You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@htrace.apache.org by Chunxu Tang <ch...@gmail.com> on 2015/02/11 16:55:02 UTC

Trace HBase/HDFS with HTrace

Hi all,

Now I’m exploiting HTrace to trace request level data flows in HBase and
HDFS. I have successfully traced HBase and HDFS by using HTrace,
respectively.

After that, I combine HBase and HDFS together and I want to just send a
PUT/GET request to HBase, but to trace the whole data flow in both HBase
and HDFS. In my opinion, when I send a request such as Get to HBase, it
will at last try to read the blocks on HDFS, so I can construct a whole
data flow tracing through HBase and HDFS. While, the fact is that I can
only get tracing data of HBase, with no data of HDFS.

Could you give me any suggestions on how to trace the data flow in both
HBase and HDFS? Does anyone have similar experience? Do I need to modify
the source code? And maybe which part(s) should I touch? If I need to
modify the code, I will try to create a patch for that.

Thank you.

My Configurations:
Hadoop version: 2.6.0
HBase version: 0.99.2
HTrace version: htrace-master
OS: Ubuntu 12.04


Joshua

Re: Trace HBase/HDFS with HTrace

Posted by Masatake Iwasaki <iw...@oss.nttdata.co.jp>.
 > Attached image is generated by htrace-hbase (fixed for hbase-1).

You can see the image here...
http://iwasakims.github.io/images/end2end-create-table.png


I could not expected result for put and scan.
http://iwasakims.github.io/images/tracing-put.png
http://iwasakims.github.io/images/tracing-scan.png


On 2/27/15 07:06, Masatake Iwasaki wrote:
> I got expected end-to-end trace at least for creating hbase table by 
> htrace-3.1.0-incubating.
> Attached image is generated by htrace-hbase (fixed for hbase-1).
>
>
> On 2/27/15 06:15, Masatake Iwasaki wrote:
>> > The double-detach should be fixed in HBase as well.
>>
>> I got HDFS tracing spans originated by HBase operation
>> with htrace-3.2.0-SNAPSHOT (with HTRACE-119)
>> but there seems to be the case that span is not closed in HBase.
>> I did not get RuntimeException due to double detach/close (by 
>> HTRACE-101).
>>
>>
>> On 2/27/15 00:42, Nick Dimiduk wrote:
>>> The double-detach should be fixed in HBase as well.
>>>
>>> On Thu, Feb 26, 2015 at 1:07 AM, Masatake Iwasaki <
>>> iwasakims@oss.nttdata.co.jp> wrote:
>>>
>>>>> perhaps a double detach.
>>>> It turned out to be double detach of NullScope singleton and I filed
>>>> HTRACE-119.
>>>> I got the error on meta assignments on regionserver startup without
>>>> starting trace spans..
>>>>
>>>> Thanks,
>>>> Masatake
>>>>
>>>>
>>>> On 2/26/15 09:58, Colin P. McCabe wrote:
>>>>
>>>>> Hmm.  Looking at that error, my guess would be that there is an
>>>>> incorrect usage of TraceScope#detach going on somewhere in hbase...
>>>>> perhaps a double detach.  But I could be wrong.  We added some code
>>>>> recently to catch issues like this.
>>>>>
>>>>> best,
>>>>> Colin
>>>>>
>>>>> On Wed, Feb 25, 2015 at 12:28 AM, Masatake Iwasaki
>>>>> <iw...@oss.nttdata.co.jp> wrote:
>>>>>
>>>>>> I tried hbase-1 built against today's htrace-3.2.0-SNAPSHOT (with 
>>>>>> quick
>>>>>> fix
>>>>>> to TestHTraceHooks).
>>>>>> I got the error below in regionserver log.
>>>>>> I will dig this tomorrow.::
>>>>>>
>>>>>>     2015-02-25 00:18:29,270 ERROR [RS_OPEN_META-centos7:16201-0]
>>>>>> htrace.Tracer: Tried to detach trace span null but it has already 
>>>>>> been
>>>>>> detached.
>>>>>>     2015-02-25 00:18:29,271 ERROR [RS_OPEN_META-centos7:16201-0]
>>>>>> handler.OpenRegionHandler: Failed open of region=hbase:meta,,1.
>>>>>> 1588230740,
>>>>>> starting to roll back the global memstore size.
>>>>>>     java.lang.RuntimeException: Tried to detach trace span null 
>>>>>> but it has
>>>>>> already been detached.
>>>>>>             at org.apache.htrace.Tracer.clientError(Tracer.java:61)
>>>>>>             at 
>>>>>> org.apache.htrace.TraceScope.detach(TraceScope.java:57)
>>>>>>             at
>>>>>> org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:1559) 
>>>>>>
>>>>>>             at
>>>>>> org.apache.hadoop.hbase.regionserver.wal.WALUtil.writeRegionEventMarker( 
>>>>>>
>>>>>> WALUtil.java:94)
>>>>>>             at
>>>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>>>> writeRegionOpenMarker(HRegion.java:910)
>>>>>>             at
>>>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>>>> openHRegion(HRegion.java:4911)
>>>>>>             at
>>>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>>>> openHRegion(HRegion.java:4874)
>>>>>>             at
>>>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>>>> openHRegion(HRegion.java:4845)
>>>>>>             at
>>>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>>>> openHRegion(HRegion.java:4801)
>>>>>>             at
>>>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>>>> openHRegion(HRegion.java:4752)
>>>>>>             at
>>>>>> org.apache.hadoop.hbase.regionserver.handler.
>>>>>> OpenRegionHandler.openRegion(OpenRegionHandler.java:356)
>>>>>>             at
>>>>>> org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.process( 
>>>>>>
>>>>>> OpenRegionHandler.java:126)
>>>>>>             at
>>>>>> org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:128) 
>>>>>>
>>>>>>             at
>>>>>> java.util.concurrent.ThreadPoolExecutor.runWorker(
>>>>>> ThreadPoolExecutor.java:1145)
>>>>>>             at
>>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(
>>>>>> ThreadPoolExecutor.java:615)
>>>>>>             at java.lang.Thread.run(Thread.java:745)
>>>>>>
>>>>>>
>>>>>>
>>>>>> On 2/24/15 18:27, Colin P. McCabe wrote:
>>>>>>
>>>>>>> Thanks for trying this, Mastake. I've got HDFS working on my 
>>>>>>> cluster
>>>>>>> with tracing and LocalFileSpanReceiver.  Did you try using HBase +
>>>>>>> HDFS with LocalFileSpanReceiver?  Be sure to use a build including
>>>>>>> HTRACE-112 since LFSR was kind of busted prior to that.
>>>>>>>
>>>>>>> I'm going to do a longer writeup about getting HDFS + HBase working
>>>>>>> with other span receivers just as soon as I finish stomping a 
>>>>>>> few more
>>>>>>> bugs.
>>>>>>>
>>>>>>> best,
>>>>>>> Colin
>>>>>>>
>>>>>>> On Tue, Feb 24, 2015 at 12:04 PM, Masatake Iwasaki
>>>>>>> <iw...@oss.nttdata.co.jp> wrote:
>>>>>>>
>>>>>>>> Hi,
>>>>>>>>
>>>>>>>> Thanks for trying this. I am sorry for late reply.
>>>>>>>>
>>>>>>>> I tried this today
>>>>>>>> by hbase-1.0.1-SANPSHOT built with
>>>>>>>> {{-Dhadoop-two.version=2.7.0-SNAPSHOT}}
>>>>>>>> in pseudo distributed cluster
>>>>>>>> but failed to get end-to-end trace.
>>>>>>>>
>>>>>>>> I checked that
>>>>>>>> * tracing works for both of hbase and hdfs,
>>>>>>>> * hbase runs with 2.7.0-SNAPSHOT jar of hadoop.
>>>>>>>>
>>>>>>>> When I did do put with tracing on,
>>>>>>>> I saw span named "FSHLog.sync" with annotations such as
>>>>>>>> "syncing writer" and "writer synced".
>>>>>>>> The code for tracing in FSHLog worked at least.
>>>>>>>>
>>>>>>>> I'm still looking into this.
>>>>>>>> If it turned out that tracing spans are not reached to
>>>>>>>> actual HDFS writer thread in HBase, I will file a JIRA.
>>>>>>>>
>>>>>>>> # We need hadoop-2.6.0 or higher in order to trace HDFS.
>>>>>>>> # Building hbase from source with {{-Dhadoop-two.version=2.6.0}}
>>>>>>>> # is straight forward way to do this
>>>>>>>> # because the binary release of hbase-1.0.0 bundles 
>>>>>>>> hadoop-2.5.1 jars.
>>>>>>>>
>>>>>>>> Masatake
>>>>>>>>
>>>>>>>>
>>>>>>>> On 2/11/15 08:56, Nick Dimiduk wrote:
>>>>>>>>
>>>>>>>>> Hi Joshua,
>>>>>>>>>
>>>>>>>>> In theory there's nothing special for you to do. Just issue 
>>>>>>>>> your query
>>>>>>>>> to
>>>>>>>>> HBase with tracing enabled. The active span will go through 
>>>>>>>>> HBase,
>>>>>>>>> down
>>>>>>>>> into HDFS, and back again. You'll need both systems collecting 
>>>>>>>>> spans
>>>>>>>>> into
>>>>>>>>> the same place so that you can report on the complete trace tree.
>>>>>>>>>
>>>>>>>>> I've not recently tested the end-to-end, but I believe it's 
>>>>>>>>> all there.
>>>>>>>>> If
>>>>>>>>> not, it's a bug -- this is an intended use case. Can you give 
>>>>>>>>> it a try
>>>>>>>>> and let us know how it goes?
>>>>>>>>>
>>>>>>>>> FYI, 0.99.x are preview releases of HBase and not for 
>>>>>>>>> production use.
>>>>>>>>> Just
>>>>>>>>> so you know :)
>>>>>>>>>
>>>>>>>>> -n
>>>>>>>>>
>>>>>>>>> On Wednesday, February 11, 2015, Chunxu Tang 
>>>>>>>>> <ch...@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>   Hi all,
>>>>>>>>>> Now I’m exploiting HTrace to trace request level data flows 
>>>>>>>>>> in HBase
>>>>>>>>>> and
>>>>>>>>>> HDFS. I have successfully traced HBase and HDFS by using HTrace,
>>>>>>>>>> respectively.
>>>>>>>>>>
>>>>>>>>>> After that, I combine HBase and HDFS together and I want to just
>>>>>>>>>> send a
>>>>>>>>>> PUT/GET request to HBase, but to trace the whole data flow in 
>>>>>>>>>> both
>>>>>>>>>> HBase
>>>>>>>>>> and HDFS. In my opinion, when I send a request such as Get to 
>>>>>>>>>> HBase,
>>>>>>>>>> it
>>>>>>>>>> will at last try to read the blocks on HDFS, so I can 
>>>>>>>>>> construct a
>>>>>>>>>> whole
>>>>>>>>>> data flow tracing through HBase and HDFS. While, the fact is 
>>>>>>>>>> that I
>>>>>>>>>> can
>>>>>>>>>> only get tracing data of HBase, with no data of HDFS.
>>>>>>>>>>
>>>>>>>>>> Could you give me any suggestions on how to trace the data 
>>>>>>>>>> flow in
>>>>>>>>>> both
>>>>>>>>>> HBase and HDFS? Does anyone have similar experience? Do I 
>>>>>>>>>> need to
>>>>>>>>>> modify
>>>>>>>>>> the source code? And maybe which part(s) should I touch? If I 
>>>>>>>>>> need to
>>>>>>>>>> modify the code, I will try to create a patch for that.
>>>>>>>>>>
>>>>>>>>>> Thank you.
>>>>>>>>>>
>>>>>>>>>> My Configurations:
>>>>>>>>>> Hadoop version: 2.6.0
>>>>>>>>>> HBase version: 0.99.2
>>>>>>>>>> HTrace version: htrace-master
>>>>>>>>>> OS: Ubuntu 12.04
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Joshua
>>>>>>>>>>
>>>>>>>>>>
>>
>


Re: Trace HBase/HDFS with HTrace

Posted by Masatake Iwasaki <iw...@oss.nttdata.co.jp>.
I got expected end-to-end trace at least for creating hbase table by 
htrace-3.1.0-incubating.
Attached image is generated by htrace-hbase (fixed for hbase-1).


On 2/27/15 06:15, Masatake Iwasaki wrote:
> > The double-detach should be fixed in HBase as well.
>
> I got HDFS tracing spans originated by HBase operation
> with htrace-3.2.0-SNAPSHOT (with HTRACE-119)
> but there seems to be the case that span is not closed in HBase.
> I did not get RuntimeException due to double detach/close (by 
> HTRACE-101).
>
>
> On 2/27/15 00:42, Nick Dimiduk wrote:
>> The double-detach should be fixed in HBase as well.
>>
>> On Thu, Feb 26, 2015 at 1:07 AM, Masatake Iwasaki <
>> iwasakims@oss.nttdata.co.jp> wrote:
>>
>>>> perhaps a double detach.
>>> It turned out to be double detach of NullScope singleton and I filed
>>> HTRACE-119.
>>> I got the error on meta assignments on regionserver startup without
>>> starting trace spans..
>>>
>>> Thanks,
>>> Masatake
>>>
>>>
>>> On 2/26/15 09:58, Colin P. McCabe wrote:
>>>
>>>> Hmm.  Looking at that error, my guess would be that there is an
>>>> incorrect usage of TraceScope#detach going on somewhere in hbase...
>>>> perhaps a double detach.  But I could be wrong.  We added some code
>>>> recently to catch issues like this.
>>>>
>>>> best,
>>>> Colin
>>>>
>>>> On Wed, Feb 25, 2015 at 12:28 AM, Masatake Iwasaki
>>>> <iw...@oss.nttdata.co.jp> wrote:
>>>>
>>>>> I tried hbase-1 built against today's htrace-3.2.0-SNAPSHOT (with 
>>>>> quick
>>>>> fix
>>>>> to TestHTraceHooks).
>>>>> I got the error below in regionserver log.
>>>>> I will dig this tomorrow.::
>>>>>
>>>>>     2015-02-25 00:18:29,270 ERROR [RS_OPEN_META-centos7:16201-0]
>>>>> htrace.Tracer: Tried to detach trace span null but it has already 
>>>>> been
>>>>> detached.
>>>>>     2015-02-25 00:18:29,271 ERROR [RS_OPEN_META-centos7:16201-0]
>>>>> handler.OpenRegionHandler: Failed open of region=hbase:meta,,1.
>>>>> 1588230740,
>>>>> starting to roll back the global memstore size.
>>>>>     java.lang.RuntimeException: Tried to detach trace span null 
>>>>> but it has
>>>>> already been detached.
>>>>>             at org.apache.htrace.Tracer.clientError(Tracer.java:61)
>>>>>             at 
>>>>> org.apache.htrace.TraceScope.detach(TraceScope.java:57)
>>>>>             at
>>>>> org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:1559) 
>>>>>
>>>>>             at
>>>>> org.apache.hadoop.hbase.regionserver.wal.WALUtil.writeRegionEventMarker( 
>>>>>
>>>>> WALUtil.java:94)
>>>>>             at
>>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>>> writeRegionOpenMarker(HRegion.java:910)
>>>>>             at
>>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>>> openHRegion(HRegion.java:4911)
>>>>>             at
>>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>>> openHRegion(HRegion.java:4874)
>>>>>             at
>>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>>> openHRegion(HRegion.java:4845)
>>>>>             at
>>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>>> openHRegion(HRegion.java:4801)
>>>>>             at
>>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>>> openHRegion(HRegion.java:4752)
>>>>>             at
>>>>> org.apache.hadoop.hbase.regionserver.handler.
>>>>> OpenRegionHandler.openRegion(OpenRegionHandler.java:356)
>>>>>             at
>>>>> org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.process( 
>>>>>
>>>>> OpenRegionHandler.java:126)
>>>>>             at
>>>>> org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:128) 
>>>>>
>>>>>             at
>>>>> java.util.concurrent.ThreadPoolExecutor.runWorker(
>>>>> ThreadPoolExecutor.java:1145)
>>>>>             at
>>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(
>>>>> ThreadPoolExecutor.java:615)
>>>>>             at java.lang.Thread.run(Thread.java:745)
>>>>>
>>>>>
>>>>>
>>>>> On 2/24/15 18:27, Colin P. McCabe wrote:
>>>>>
>>>>>> Thanks for trying this, Mastake. I've got HDFS working on my cluster
>>>>>> with tracing and LocalFileSpanReceiver.  Did you try using HBase +
>>>>>> HDFS with LocalFileSpanReceiver?  Be sure to use a build including
>>>>>> HTRACE-112 since LFSR was kind of busted prior to that.
>>>>>>
>>>>>> I'm going to do a longer writeup about getting HDFS + HBase working
>>>>>> with other span receivers just as soon as I finish stomping a few 
>>>>>> more
>>>>>> bugs.
>>>>>>
>>>>>> best,
>>>>>> Colin
>>>>>>
>>>>>> On Tue, Feb 24, 2015 at 12:04 PM, Masatake Iwasaki
>>>>>> <iw...@oss.nttdata.co.jp> wrote:
>>>>>>
>>>>>>> Hi,
>>>>>>>
>>>>>>> Thanks for trying this. I am sorry for late reply.
>>>>>>>
>>>>>>> I tried this today
>>>>>>> by hbase-1.0.1-SANPSHOT built with
>>>>>>> {{-Dhadoop-two.version=2.7.0-SNAPSHOT}}
>>>>>>> in pseudo distributed cluster
>>>>>>> but failed to get end-to-end trace.
>>>>>>>
>>>>>>> I checked that
>>>>>>> * tracing works for both of hbase and hdfs,
>>>>>>> * hbase runs with 2.7.0-SNAPSHOT jar of hadoop.
>>>>>>>
>>>>>>> When I did do put with tracing on,
>>>>>>> I saw span named "FSHLog.sync" with annotations such as
>>>>>>> "syncing writer" and "writer synced".
>>>>>>> The code for tracing in FSHLog worked at least.
>>>>>>>
>>>>>>> I'm still looking into this.
>>>>>>> If it turned out that tracing spans are not reached to
>>>>>>> actual HDFS writer thread in HBase, I will file a JIRA.
>>>>>>>
>>>>>>> # We need hadoop-2.6.0 or higher in order to trace HDFS.
>>>>>>> # Building hbase from source with {{-Dhadoop-two.version=2.6.0}}
>>>>>>> # is straight forward way to do this
>>>>>>> # because the binary release of hbase-1.0.0 bundles hadoop-2.5.1 
>>>>>>> jars.
>>>>>>>
>>>>>>> Masatake
>>>>>>>
>>>>>>>
>>>>>>> On 2/11/15 08:56, Nick Dimiduk wrote:
>>>>>>>
>>>>>>>> Hi Joshua,
>>>>>>>>
>>>>>>>> In theory there's nothing special for you to do. Just issue 
>>>>>>>> your query
>>>>>>>> to
>>>>>>>> HBase with tracing enabled. The active span will go through HBase,
>>>>>>>> down
>>>>>>>> into HDFS, and back again. You'll need both systems collecting 
>>>>>>>> spans
>>>>>>>> into
>>>>>>>> the same place so that you can report on the complete trace tree.
>>>>>>>>
>>>>>>>> I've not recently tested the end-to-end, but I believe it's all 
>>>>>>>> there.
>>>>>>>> If
>>>>>>>> not, it's a bug -- this is an intended use case. Can you give 
>>>>>>>> it a try
>>>>>>>> and let us know how it goes?
>>>>>>>>
>>>>>>>> FYI, 0.99.x are preview releases of HBase and not for 
>>>>>>>> production use.
>>>>>>>> Just
>>>>>>>> so you know :)
>>>>>>>>
>>>>>>>> -n
>>>>>>>>
>>>>>>>> On Wednesday, February 11, 2015, Chunxu Tang 
>>>>>>>> <ch...@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>   Hi all,
>>>>>>>>> Now I’m exploiting HTrace to trace request level data flows in 
>>>>>>>>> HBase
>>>>>>>>> and
>>>>>>>>> HDFS. I have successfully traced HBase and HDFS by using HTrace,
>>>>>>>>> respectively.
>>>>>>>>>
>>>>>>>>> After that, I combine HBase and HDFS together and I want to just
>>>>>>>>> send a
>>>>>>>>> PUT/GET request to HBase, but to trace the whole data flow in 
>>>>>>>>> both
>>>>>>>>> HBase
>>>>>>>>> and HDFS. In my opinion, when I send a request such as Get to 
>>>>>>>>> HBase,
>>>>>>>>> it
>>>>>>>>> will at last try to read the blocks on HDFS, so I can construct a
>>>>>>>>> whole
>>>>>>>>> data flow tracing through HBase and HDFS. While, the fact is 
>>>>>>>>> that I
>>>>>>>>> can
>>>>>>>>> only get tracing data of HBase, with no data of HDFS.
>>>>>>>>>
>>>>>>>>> Could you give me any suggestions on how to trace the data 
>>>>>>>>> flow in
>>>>>>>>> both
>>>>>>>>> HBase and HDFS? Does anyone have similar experience? Do I need to
>>>>>>>>> modify
>>>>>>>>> the source code? And maybe which part(s) should I touch? If I 
>>>>>>>>> need to
>>>>>>>>> modify the code, I will try to create a patch for that.
>>>>>>>>>
>>>>>>>>> Thank you.
>>>>>>>>>
>>>>>>>>> My Configurations:
>>>>>>>>> Hadoop version: 2.6.0
>>>>>>>>> HBase version: 0.99.2
>>>>>>>>> HTrace version: htrace-master
>>>>>>>>> OS: Ubuntu 12.04
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Joshua
>>>>>>>>>
>>>>>>>>>
>


Re: Trace HBase/HDFS with HTrace

Posted by Masatake Iwasaki <iw...@oss.nttdata.co.jp>.
 > The double-detach should be fixed in HBase as well.

I got HDFS tracing spans originated by HBase operation
with htrace-3.2.0-SNAPSHOT (with HTRACE-119)
but there seems to be the case that span is not closed in HBase.
I did not get RuntimeException due to double detach/close (by HTRACE-101).


On 2/27/15 00:42, Nick Dimiduk wrote:
> The double-detach should be fixed in HBase as well.
>
> On Thu, Feb 26, 2015 at 1:07 AM, Masatake Iwasaki <
> iwasakims@oss.nttdata.co.jp> wrote:
>
>>> perhaps a double detach.
>> It turned out to be double detach of NullScope singleton and I filed
>> HTRACE-119.
>> I got the error on meta assignments on regionserver startup without
>> starting trace spans..
>>
>> Thanks,
>> Masatake
>>
>>
>> On 2/26/15 09:58, Colin P. McCabe wrote:
>>
>>> Hmm.  Looking at that error, my guess would be that there is an
>>> incorrect usage of TraceScope#detach going on somewhere in hbase...
>>> perhaps a double detach.  But I could be wrong.  We added some code
>>> recently to catch issues like this.
>>>
>>> best,
>>> Colin
>>>
>>> On Wed, Feb 25, 2015 at 12:28 AM, Masatake Iwasaki
>>> <iw...@oss.nttdata.co.jp> wrote:
>>>
>>>> I tried hbase-1 built against today's htrace-3.2.0-SNAPSHOT (with quick
>>>> fix
>>>> to TestHTraceHooks).
>>>> I got the error below in regionserver log.
>>>> I will dig this tomorrow.::
>>>>
>>>>     2015-02-25 00:18:29,270 ERROR [RS_OPEN_META-centos7:16201-0]
>>>> htrace.Tracer: Tried to detach trace span null but it has already been
>>>> detached.
>>>>     2015-02-25 00:18:29,271 ERROR [RS_OPEN_META-centos7:16201-0]
>>>> handler.OpenRegionHandler: Failed open of region=hbase:meta,,1.
>>>> 1588230740,
>>>> starting to roll back the global memstore size.
>>>>     java.lang.RuntimeException: Tried to detach trace span null but it has
>>>> already been detached.
>>>>             at org.apache.htrace.Tracer.clientError(Tracer.java:61)
>>>>             at org.apache.htrace.TraceScope.detach(TraceScope.java:57)
>>>>             at
>>>> org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:1559)
>>>>             at
>>>> org.apache.hadoop.hbase.regionserver.wal.WALUtil.writeRegionEventMarker(
>>>> WALUtil.java:94)
>>>>             at
>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>> writeRegionOpenMarker(HRegion.java:910)
>>>>             at
>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>> openHRegion(HRegion.java:4911)
>>>>             at
>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>> openHRegion(HRegion.java:4874)
>>>>             at
>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>> openHRegion(HRegion.java:4845)
>>>>             at
>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>> openHRegion(HRegion.java:4801)
>>>>             at
>>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>>> openHRegion(HRegion.java:4752)
>>>>             at
>>>> org.apache.hadoop.hbase.regionserver.handler.
>>>> OpenRegionHandler.openRegion(OpenRegionHandler.java:356)
>>>>             at
>>>> org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.process(
>>>> OpenRegionHandler.java:126)
>>>>             at
>>>> org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:128)
>>>>             at
>>>> java.util.concurrent.ThreadPoolExecutor.runWorker(
>>>> ThreadPoolExecutor.java:1145)
>>>>             at
>>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(
>>>> ThreadPoolExecutor.java:615)
>>>>             at java.lang.Thread.run(Thread.java:745)
>>>>
>>>>
>>>>
>>>> On 2/24/15 18:27, Colin P. McCabe wrote:
>>>>
>>>>> Thanks for trying this, Mastake.  I've got HDFS working on my cluster
>>>>> with tracing and LocalFileSpanReceiver.  Did you try using HBase +
>>>>> HDFS with LocalFileSpanReceiver?  Be sure to use a build including
>>>>> HTRACE-112 since LFSR was kind of busted prior to that.
>>>>>
>>>>> I'm going to do a longer writeup about getting HDFS + HBase working
>>>>> with other span receivers just as soon as I finish stomping a few more
>>>>> bugs.
>>>>>
>>>>> best,
>>>>> Colin
>>>>>
>>>>> On Tue, Feb 24, 2015 at 12:04 PM, Masatake Iwasaki
>>>>> <iw...@oss.nttdata.co.jp> wrote:
>>>>>
>>>>>> Hi,
>>>>>>
>>>>>> Thanks for trying this. I am sorry for late reply.
>>>>>>
>>>>>> I tried this today
>>>>>> by hbase-1.0.1-SANPSHOT built with
>>>>>> {{-Dhadoop-two.version=2.7.0-SNAPSHOT}}
>>>>>> in pseudo distributed cluster
>>>>>> but failed to get end-to-end trace.
>>>>>>
>>>>>> I checked that
>>>>>> * tracing works for both of hbase and hdfs,
>>>>>> * hbase runs with 2.7.0-SNAPSHOT jar of hadoop.
>>>>>>
>>>>>> When I did do put with tracing on,
>>>>>> I saw span named "FSHLog.sync" with annotations such as
>>>>>> "syncing writer" and "writer synced".
>>>>>> The code for tracing in FSHLog worked at least.
>>>>>>
>>>>>> I'm still looking into this.
>>>>>> If it turned out that tracing spans are not reached to
>>>>>> actual HDFS writer thread in HBase, I will file a JIRA.
>>>>>>
>>>>>> # We need hadoop-2.6.0 or higher in order to trace HDFS.
>>>>>> # Building hbase from source with {{-Dhadoop-two.version=2.6.0}}
>>>>>> # is straight forward way to do this
>>>>>> # because the binary release of hbase-1.0.0 bundles hadoop-2.5.1 jars.
>>>>>>
>>>>>> Masatake
>>>>>>
>>>>>>
>>>>>> On 2/11/15 08:56, Nick Dimiduk wrote:
>>>>>>
>>>>>>> Hi Joshua,
>>>>>>>
>>>>>>> In theory there's nothing special for you to do. Just issue your query
>>>>>>> to
>>>>>>> HBase with tracing enabled. The active span will go through HBase,
>>>>>>> down
>>>>>>> into HDFS, and back again. You'll need both systems collecting spans
>>>>>>> into
>>>>>>> the same place so that you can report on the complete trace tree.
>>>>>>>
>>>>>>> I've not recently tested the end-to-end, but I believe it's all there.
>>>>>>> If
>>>>>>> not, it's a bug -- this is an intended use case. Can you give it a try
>>>>>>> and let us know how it goes?
>>>>>>>
>>>>>>> FYI, 0.99.x are preview releases of HBase and not for production use.
>>>>>>> Just
>>>>>>> so you know :)
>>>>>>>
>>>>>>> -n
>>>>>>>
>>>>>>> On Wednesday, February 11, 2015, Chunxu Tang <ch...@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>   Hi all,
>>>>>>>> Now I’m exploiting HTrace to trace request level data flows in HBase
>>>>>>>> and
>>>>>>>> HDFS. I have successfully traced HBase and HDFS by using HTrace,
>>>>>>>> respectively.
>>>>>>>>
>>>>>>>> After that, I combine HBase and HDFS together and I want to just
>>>>>>>> send a
>>>>>>>> PUT/GET request to HBase, but to trace the whole data flow in both
>>>>>>>> HBase
>>>>>>>> and HDFS. In my opinion, when I send a request such as Get to HBase,
>>>>>>>> it
>>>>>>>> will at last try to read the blocks on HDFS, so I can construct a
>>>>>>>> whole
>>>>>>>> data flow tracing through HBase and HDFS. While, the fact is that I
>>>>>>>> can
>>>>>>>> only get tracing data of HBase, with no data of HDFS.
>>>>>>>>
>>>>>>>> Could you give me any suggestions on how to trace the data flow in
>>>>>>>> both
>>>>>>>> HBase and HDFS? Does anyone have similar experience? Do I need to
>>>>>>>> modify
>>>>>>>> the source code? And maybe which part(s) should I touch? If I need to
>>>>>>>> modify the code, I will try to create a patch for that.
>>>>>>>>
>>>>>>>> Thank you.
>>>>>>>>
>>>>>>>> My Configurations:
>>>>>>>> Hadoop version: 2.6.0
>>>>>>>> HBase version: 0.99.2
>>>>>>>> HTrace version: htrace-master
>>>>>>>> OS: Ubuntu 12.04
>>>>>>>>
>>>>>>>>
>>>>>>>> Joshua
>>>>>>>>
>>>>>>>>


Re: Trace HBase/HDFS with HTrace

Posted by Nick Dimiduk <nd...@gmail.com>.
The double-detach should be fixed in HBase as well.

On Thu, Feb 26, 2015 at 1:07 AM, Masatake Iwasaki <
iwasakims@oss.nttdata.co.jp> wrote:

> > perhaps a double detach.
>
> It turned out to be double detach of NullScope singleton and I filed
> HTRACE-119.
> I got the error on meta assignments on regionserver startup without
> starting trace spans..
>
> Thanks,
> Masatake
>
>
> On 2/26/15 09:58, Colin P. McCabe wrote:
>
>> Hmm.  Looking at that error, my guess would be that there is an
>> incorrect usage of TraceScope#detach going on somewhere in hbase...
>> perhaps a double detach.  But I could be wrong.  We added some code
>> recently to catch issues like this.
>>
>> best,
>> Colin
>>
>> On Wed, Feb 25, 2015 at 12:28 AM, Masatake Iwasaki
>> <iw...@oss.nttdata.co.jp> wrote:
>>
>>> I tried hbase-1 built against today's htrace-3.2.0-SNAPSHOT (with quick
>>> fix
>>> to TestHTraceHooks).
>>> I got the error below in regionserver log.
>>> I will dig this tomorrow.::
>>>
>>>    2015-02-25 00:18:29,270 ERROR [RS_OPEN_META-centos7:16201-0]
>>> htrace.Tracer: Tried to detach trace span null but it has already been
>>> detached.
>>>    2015-02-25 00:18:29,271 ERROR [RS_OPEN_META-centos7:16201-0]
>>> handler.OpenRegionHandler: Failed open of region=hbase:meta,,1.
>>> 1588230740,
>>> starting to roll back the global memstore size.
>>>    java.lang.RuntimeException: Tried to detach trace span null but it has
>>> already been detached.
>>>            at org.apache.htrace.Tracer.clientError(Tracer.java:61)
>>>            at org.apache.htrace.TraceScope.detach(TraceScope.java:57)
>>>            at
>>> org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:1559)
>>>            at
>>> org.apache.hadoop.hbase.regionserver.wal.WALUtil.writeRegionEventMarker(
>>> WALUtil.java:94)
>>>            at
>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>> writeRegionOpenMarker(HRegion.java:910)
>>>            at
>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>> openHRegion(HRegion.java:4911)
>>>            at
>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>> openHRegion(HRegion.java:4874)
>>>            at
>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>> openHRegion(HRegion.java:4845)
>>>            at
>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>> openHRegion(HRegion.java:4801)
>>>            at
>>> org.apache.hadoop.hbase.regionserver.HRegion.
>>> openHRegion(HRegion.java:4752)
>>>            at
>>> org.apache.hadoop.hbase.regionserver.handler.
>>> OpenRegionHandler.openRegion(OpenRegionHandler.java:356)
>>>            at
>>> org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.process(
>>> OpenRegionHandler.java:126)
>>>            at
>>> org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:128)
>>>            at
>>> java.util.concurrent.ThreadPoolExecutor.runWorker(
>>> ThreadPoolExecutor.java:1145)
>>>            at
>>> java.util.concurrent.ThreadPoolExecutor$Worker.run(
>>> ThreadPoolExecutor.java:615)
>>>            at java.lang.Thread.run(Thread.java:745)
>>>
>>>
>>>
>>> On 2/24/15 18:27, Colin P. McCabe wrote:
>>>
>>>> Thanks for trying this, Mastake.  I've got HDFS working on my cluster
>>>> with tracing and LocalFileSpanReceiver.  Did you try using HBase +
>>>> HDFS with LocalFileSpanReceiver?  Be sure to use a build including
>>>> HTRACE-112 since LFSR was kind of busted prior to that.
>>>>
>>>> I'm going to do a longer writeup about getting HDFS + HBase working
>>>> with other span receivers just as soon as I finish stomping a few more
>>>> bugs.
>>>>
>>>> best,
>>>> Colin
>>>>
>>>> On Tue, Feb 24, 2015 at 12:04 PM, Masatake Iwasaki
>>>> <iw...@oss.nttdata.co.jp> wrote:
>>>>
>>>>> Hi,
>>>>>
>>>>> Thanks for trying this. I am sorry for late reply.
>>>>>
>>>>> I tried this today
>>>>> by hbase-1.0.1-SANPSHOT built with
>>>>> {{-Dhadoop-two.version=2.7.0-SNAPSHOT}}
>>>>> in pseudo distributed cluster
>>>>> but failed to get end-to-end trace.
>>>>>
>>>>> I checked that
>>>>> * tracing works for both of hbase and hdfs,
>>>>> * hbase runs with 2.7.0-SNAPSHOT jar of hadoop.
>>>>>
>>>>> When I did do put with tracing on,
>>>>> I saw span named "FSHLog.sync" with annotations such as
>>>>> "syncing writer" and "writer synced".
>>>>> The code for tracing in FSHLog worked at least.
>>>>>
>>>>> I'm still looking into this.
>>>>> If it turned out that tracing spans are not reached to
>>>>> actual HDFS writer thread in HBase, I will file a JIRA.
>>>>>
>>>>> # We need hadoop-2.6.0 or higher in order to trace HDFS.
>>>>> # Building hbase from source with {{-Dhadoop-two.version=2.6.0}}
>>>>> # is straight forward way to do this
>>>>> # because the binary release of hbase-1.0.0 bundles hadoop-2.5.1 jars.
>>>>>
>>>>> Masatake
>>>>>
>>>>>
>>>>> On 2/11/15 08:56, Nick Dimiduk wrote:
>>>>>
>>>>>> Hi Joshua,
>>>>>>
>>>>>> In theory there's nothing special for you to do. Just issue your query
>>>>>> to
>>>>>> HBase with tracing enabled. The active span will go through HBase,
>>>>>> down
>>>>>> into HDFS, and back again. You'll need both systems collecting spans
>>>>>> into
>>>>>> the same place so that you can report on the complete trace tree.
>>>>>>
>>>>>> I've not recently tested the end-to-end, but I believe it's all there.
>>>>>> If
>>>>>> not, it's a bug -- this is an intended use case. Can you give it a try
>>>>>> and let us know how it goes?
>>>>>>
>>>>>> FYI, 0.99.x are preview releases of HBase and not for production use.
>>>>>> Just
>>>>>> so you know :)
>>>>>>
>>>>>> -n
>>>>>>
>>>>>> On Wednesday, February 11, 2015, Chunxu Tang <ch...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>  Hi all,
>>>>>>>
>>>>>>> Now I’m exploiting HTrace to trace request level data flows in HBase
>>>>>>> and
>>>>>>> HDFS. I have successfully traced HBase and HDFS by using HTrace,
>>>>>>> respectively.
>>>>>>>
>>>>>>> After that, I combine HBase and HDFS together and I want to just
>>>>>>> send a
>>>>>>> PUT/GET request to HBase, but to trace the whole data flow in both
>>>>>>> HBase
>>>>>>> and HDFS. In my opinion, when I send a request such as Get to HBase,
>>>>>>> it
>>>>>>> will at last try to read the blocks on HDFS, so I can construct a
>>>>>>> whole
>>>>>>> data flow tracing through HBase and HDFS. While, the fact is that I
>>>>>>> can
>>>>>>> only get tracing data of HBase, with no data of HDFS.
>>>>>>>
>>>>>>> Could you give me any suggestions on how to trace the data flow in
>>>>>>> both
>>>>>>> HBase and HDFS? Does anyone have similar experience? Do I need to
>>>>>>> modify
>>>>>>> the source code? And maybe which part(s) should I touch? If I need to
>>>>>>> modify the code, I will try to create a patch for that.
>>>>>>>
>>>>>>> Thank you.
>>>>>>>
>>>>>>> My Configurations:
>>>>>>> Hadoop version: 2.6.0
>>>>>>> HBase version: 0.99.2
>>>>>>> HTrace version: htrace-master
>>>>>>> OS: Ubuntu 12.04
>>>>>>>
>>>>>>>
>>>>>>> Joshua
>>>>>>>
>>>>>>>
>

Re: Trace HBase/HDFS with HTrace

Posted by Masatake Iwasaki <iw...@oss.nttdata.co.jp>.
 > perhaps a double detach.

It turned out to be double detach of NullScope singleton and I filed 
HTRACE-119.
I got the error on meta assignments on regionserver startup without 
starting trace spans..

Thanks,
Masatake

On 2/26/15 09:58, Colin P. McCabe wrote:
> Hmm.  Looking at that error, my guess would be that there is an
> incorrect usage of TraceScope#detach going on somewhere in hbase...
> perhaps a double detach.  But I could be wrong.  We added some code
> recently to catch issues like this.
>
> best,
> Colin
>
> On Wed, Feb 25, 2015 at 12:28 AM, Masatake Iwasaki
> <iw...@oss.nttdata.co.jp> wrote:
>> I tried hbase-1 built against today's htrace-3.2.0-SNAPSHOT (with quick fix
>> to TestHTraceHooks).
>> I got the error below in regionserver log.
>> I will dig this tomorrow.::
>>
>>    2015-02-25 00:18:29,270 ERROR [RS_OPEN_META-centos7:16201-0]
>> htrace.Tracer: Tried to detach trace span null but it has already been
>> detached.
>>    2015-02-25 00:18:29,271 ERROR [RS_OPEN_META-centos7:16201-0]
>> handler.OpenRegionHandler: Failed open of region=hbase:meta,,1.1588230740,
>> starting to roll back the global memstore size.
>>    java.lang.RuntimeException: Tried to detach trace span null but it has
>> already been detached.
>>            at org.apache.htrace.Tracer.clientError(Tracer.java:61)
>>            at org.apache.htrace.TraceScope.detach(TraceScope.java:57)
>>            at
>> org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:1559)
>>            at
>> org.apache.hadoop.hbase.regionserver.wal.WALUtil.writeRegionEventMarker(WALUtil.java:94)
>>            at
>> org.apache.hadoop.hbase.regionserver.HRegion.writeRegionOpenMarker(HRegion.java:910)
>>            at
>> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:4911)
>>            at
>> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:4874)
>>            at
>> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:4845)
>>            at
>> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:4801)
>>            at
>> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:4752)
>>            at
>> org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.openRegion(OpenRegionHandler.java:356)
>>            at
>> org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.process(OpenRegionHandler.java:126)
>>            at
>> org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:128)
>>            at
>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>>            at
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>>            at java.lang.Thread.run(Thread.java:745)
>>
>>
>>
>> On 2/24/15 18:27, Colin P. McCabe wrote:
>>> Thanks for trying this, Mastake.  I've got HDFS working on my cluster
>>> with tracing and LocalFileSpanReceiver.  Did you try using HBase +
>>> HDFS with LocalFileSpanReceiver?  Be sure to use a build including
>>> HTRACE-112 since LFSR was kind of busted prior to that.
>>>
>>> I'm going to do a longer writeup about getting HDFS + HBase working
>>> with other span receivers just as soon as I finish stomping a few more
>>> bugs.
>>>
>>> best,
>>> Colin
>>>
>>> On Tue, Feb 24, 2015 at 12:04 PM, Masatake Iwasaki
>>> <iw...@oss.nttdata.co.jp> wrote:
>>>> Hi,
>>>>
>>>> Thanks for trying this. I am sorry for late reply.
>>>>
>>>> I tried this today
>>>> by hbase-1.0.1-SANPSHOT built with
>>>> {{-Dhadoop-two.version=2.7.0-SNAPSHOT}}
>>>> in pseudo distributed cluster
>>>> but failed to get end-to-end trace.
>>>>
>>>> I checked that
>>>> * tracing works for both of hbase and hdfs,
>>>> * hbase runs with 2.7.0-SNAPSHOT jar of hadoop.
>>>>
>>>> When I did do put with tracing on,
>>>> I saw span named "FSHLog.sync" with annotations such as
>>>> "syncing writer" and "writer synced".
>>>> The code for tracing in FSHLog worked at least.
>>>>
>>>> I'm still looking into this.
>>>> If it turned out that tracing spans are not reached to
>>>> actual HDFS writer thread in HBase, I will file a JIRA.
>>>>
>>>> # We need hadoop-2.6.0 or higher in order to trace HDFS.
>>>> # Building hbase from source with {{-Dhadoop-two.version=2.6.0}}
>>>> # is straight forward way to do this
>>>> # because the binary release of hbase-1.0.0 bundles hadoop-2.5.1 jars.
>>>>
>>>> Masatake
>>>>
>>>>
>>>> On 2/11/15 08:56, Nick Dimiduk wrote:
>>>>> Hi Joshua,
>>>>>
>>>>> In theory there's nothing special for you to do. Just issue your query
>>>>> to
>>>>> HBase with tracing enabled. The active span will go through HBase, down
>>>>> into HDFS, and back again. You'll need both systems collecting spans
>>>>> into
>>>>> the same place so that you can report on the complete trace tree.
>>>>>
>>>>> I've not recently tested the end-to-end, but I believe it's all there.
>>>>> If
>>>>> not, it's a bug -- this is an intended use case. Can you give it a try
>>>>> and let us know how it goes?
>>>>>
>>>>> FYI, 0.99.x are preview releases of HBase and not for production use.
>>>>> Just
>>>>> so you know :)
>>>>>
>>>>> -n
>>>>>
>>>>> On Wednesday, February 11, 2015, Chunxu Tang <ch...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Hi all,
>>>>>>
>>>>>> Now I’m exploiting HTrace to trace request level data flows in HBase
>>>>>> and
>>>>>> HDFS. I have successfully traced HBase and HDFS by using HTrace,
>>>>>> respectively.
>>>>>>
>>>>>> After that, I combine HBase and HDFS together and I want to just send a
>>>>>> PUT/GET request to HBase, but to trace the whole data flow in both
>>>>>> HBase
>>>>>> and HDFS. In my opinion, when I send a request such as Get to HBase, it
>>>>>> will at last try to read the blocks on HDFS, so I can construct a whole
>>>>>> data flow tracing through HBase and HDFS. While, the fact is that I can
>>>>>> only get tracing data of HBase, with no data of HDFS.
>>>>>>
>>>>>> Could you give me any suggestions on how to trace the data flow in both
>>>>>> HBase and HDFS? Does anyone have similar experience? Do I need to
>>>>>> modify
>>>>>> the source code? And maybe which part(s) should I touch? If I need to
>>>>>> modify the code, I will try to create a patch for that.
>>>>>>
>>>>>> Thank you.
>>>>>>
>>>>>> My Configurations:
>>>>>> Hadoop version: 2.6.0
>>>>>> HBase version: 0.99.2
>>>>>> HTrace version: htrace-master
>>>>>> OS: Ubuntu 12.04
>>>>>>
>>>>>>
>>>>>> Joshua
>>>>>>


Re: Trace HBase/HDFS with HTrace

Posted by "Colin P. McCabe" <cm...@apache.org>.
Hmm.  Looking at that error, my guess would be that there is an
incorrect usage of TraceScope#detach going on somewhere in hbase...
perhaps a double detach.  But I could be wrong.  We added some code
recently to catch issues like this.

best,
Colin

On Wed, Feb 25, 2015 at 12:28 AM, Masatake Iwasaki
<iw...@oss.nttdata.co.jp> wrote:
> I tried hbase-1 built against today's htrace-3.2.0-SNAPSHOT (with quick fix
> to TestHTraceHooks).
> I got the error below in regionserver log.
> I will dig this tomorrow.::
>
>   2015-02-25 00:18:29,270 ERROR [RS_OPEN_META-centos7:16201-0]
> htrace.Tracer: Tried to detach trace span null but it has already been
> detached.
>   2015-02-25 00:18:29,271 ERROR [RS_OPEN_META-centos7:16201-0]
> handler.OpenRegionHandler: Failed open of region=hbase:meta,,1.1588230740,
> starting to roll back the global memstore size.
>   java.lang.RuntimeException: Tried to detach trace span null but it has
> already been detached.
>           at org.apache.htrace.Tracer.clientError(Tracer.java:61)
>           at org.apache.htrace.TraceScope.detach(TraceScope.java:57)
>           at
> org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:1559)
>           at
> org.apache.hadoop.hbase.regionserver.wal.WALUtil.writeRegionEventMarker(WALUtil.java:94)
>           at
> org.apache.hadoop.hbase.regionserver.HRegion.writeRegionOpenMarker(HRegion.java:910)
>           at
> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:4911)
>           at
> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:4874)
>           at
> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:4845)
>           at
> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:4801)
>           at
> org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:4752)
>           at
> org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.openRegion(OpenRegionHandler.java:356)
>           at
> org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.process(OpenRegionHandler.java:126)
>           at
> org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:128)
>           at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>           at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>           at java.lang.Thread.run(Thread.java:745)
>
>
>
> On 2/24/15 18:27, Colin P. McCabe wrote:
>>
>> Thanks for trying this, Mastake.  I've got HDFS working on my cluster
>> with tracing and LocalFileSpanReceiver.  Did you try using HBase +
>> HDFS with LocalFileSpanReceiver?  Be sure to use a build including
>> HTRACE-112 since LFSR was kind of busted prior to that.
>>
>> I'm going to do a longer writeup about getting HDFS + HBase working
>> with other span receivers just as soon as I finish stomping a few more
>> bugs.
>>
>> best,
>> Colin
>>
>> On Tue, Feb 24, 2015 at 12:04 PM, Masatake Iwasaki
>> <iw...@oss.nttdata.co.jp> wrote:
>>>
>>> Hi,
>>>
>>> Thanks for trying this. I am sorry for late reply.
>>>
>>> I tried this today
>>> by hbase-1.0.1-SANPSHOT built with
>>> {{-Dhadoop-two.version=2.7.0-SNAPSHOT}}
>>> in pseudo distributed cluster
>>> but failed to get end-to-end trace.
>>>
>>> I checked that
>>> * tracing works for both of hbase and hdfs,
>>> * hbase runs with 2.7.0-SNAPSHOT jar of hadoop.
>>>
>>> When I did do put with tracing on,
>>> I saw span named "FSHLog.sync" with annotations such as
>>> "syncing writer" and "writer synced".
>>> The code for tracing in FSHLog worked at least.
>>>
>>> I'm still looking into this.
>>> If it turned out that tracing spans are not reached to
>>> actual HDFS writer thread in HBase, I will file a JIRA.
>>>
>>> # We need hadoop-2.6.0 or higher in order to trace HDFS.
>>> # Building hbase from source with {{-Dhadoop-two.version=2.6.0}}
>>> # is straight forward way to do this
>>> # because the binary release of hbase-1.0.0 bundles hadoop-2.5.1 jars.
>>>
>>> Masatake
>>>
>>>
>>> On 2/11/15 08:56, Nick Dimiduk wrote:
>>>>
>>>> Hi Joshua,
>>>>
>>>> In theory there's nothing special for you to do. Just issue your query
>>>> to
>>>> HBase with tracing enabled. The active span will go through HBase, down
>>>> into HDFS, and back again. You'll need both systems collecting spans
>>>> into
>>>> the same place so that you can report on the complete trace tree.
>>>>
>>>> I've not recently tested the end-to-end, but I believe it's all there.
>>>> If
>>>> not, it's a bug -- this is an intended use case. Can you give it a try
>>>> and let us know how it goes?
>>>>
>>>> FYI, 0.99.x are preview releases of HBase and not for production use.
>>>> Just
>>>> so you know :)
>>>>
>>>> -n
>>>>
>>>> On Wednesday, February 11, 2015, Chunxu Tang <ch...@gmail.com>
>>>> wrote:
>>>>
>>>>> Hi all,
>>>>>
>>>>> Now I’m exploiting HTrace to trace request level data flows in HBase
>>>>> and
>>>>> HDFS. I have successfully traced HBase and HDFS by using HTrace,
>>>>> respectively.
>>>>>
>>>>> After that, I combine HBase and HDFS together and I want to just send a
>>>>> PUT/GET request to HBase, but to trace the whole data flow in both
>>>>> HBase
>>>>> and HDFS. In my opinion, when I send a request such as Get to HBase, it
>>>>> will at last try to read the blocks on HDFS, so I can construct a whole
>>>>> data flow tracing through HBase and HDFS. While, the fact is that I can
>>>>> only get tracing data of HBase, with no data of HDFS.
>>>>>
>>>>> Could you give me any suggestions on how to trace the data flow in both
>>>>> HBase and HDFS? Does anyone have similar experience? Do I need to
>>>>> modify
>>>>> the source code? And maybe which part(s) should I touch? If I need to
>>>>> modify the code, I will try to create a patch for that.
>>>>>
>>>>> Thank you.
>>>>>
>>>>> My Configurations:
>>>>> Hadoop version: 2.6.0
>>>>> HBase version: 0.99.2
>>>>> HTrace version: htrace-master
>>>>> OS: Ubuntu 12.04
>>>>>
>>>>>
>>>>> Joshua
>>>>>
>

Re: Trace HBase/HDFS with HTrace

Posted by Masatake Iwasaki <iw...@oss.nttdata.co.jp>.
I tried hbase-1 built against today's htrace-3.2.0-SNAPSHOT (with quick 
fix to TestHTraceHooks).
I got the error below in regionserver log.
I will dig this tomorrow.::

   2015-02-25 00:18:29,270 ERROR [RS_OPEN_META-centos7:16201-0] 
htrace.Tracer: Tried to detach trace span null but it has already been 
detached.
   2015-02-25 00:18:29,271 ERROR [RS_OPEN_META-centos7:16201-0] 
handler.OpenRegionHandler: Failed open of 
region=hbase:meta,,1.1588230740, starting to roll back the global 
memstore size.
   java.lang.RuntimeException: Tried to detach trace span null but it 
has already been detached.
           at org.apache.htrace.Tracer.clientError(Tracer.java:61)
           at org.apache.htrace.TraceScope.detach(TraceScope.java:57)
           at 
org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:1559)
           at 
org.apache.hadoop.hbase.regionserver.wal.WALUtil.writeRegionEventMarker(WALUtil.java:94)
           at 
org.apache.hadoop.hbase.regionserver.HRegion.writeRegionOpenMarker(HRegion.java:910)
           at 
org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:4911)
           at 
org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:4874)
           at 
org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:4845)
           at 
org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:4801)
           at 
org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:4752)
           at 
org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.openRegion(OpenRegionHandler.java:356)
           at 
org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.process(OpenRegionHandler.java:126)
           at 
org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:128)
           at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
           at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
           at java.lang.Thread.run(Thread.java:745)


On 2/24/15 18:27, Colin P. McCabe wrote:
> Thanks for trying this, Mastake.  I've got HDFS working on my cluster
> with tracing and LocalFileSpanReceiver.  Did you try using HBase +
> HDFS with LocalFileSpanReceiver?  Be sure to use a build including
> HTRACE-112 since LFSR was kind of busted prior to that.
>
> I'm going to do a longer writeup about getting HDFS + HBase working
> with other span receivers just as soon as I finish stomping a few more
> bugs.
>
> best,
> Colin
>
> On Tue, Feb 24, 2015 at 12:04 PM, Masatake Iwasaki
> <iw...@oss.nttdata.co.jp> wrote:
>> Hi,
>>
>> Thanks for trying this. I am sorry for late reply.
>>
>> I tried this today
>> by hbase-1.0.1-SANPSHOT built with {{-Dhadoop-two.version=2.7.0-SNAPSHOT}}
>> in pseudo distributed cluster
>> but failed to get end-to-end trace.
>>
>> I checked that
>> * tracing works for both of hbase and hdfs,
>> * hbase runs with 2.7.0-SNAPSHOT jar of hadoop.
>>
>> When I did do put with tracing on,
>> I saw span named "FSHLog.sync" with annotations such as
>> "syncing writer" and "writer synced".
>> The code for tracing in FSHLog worked at least.
>>
>> I'm still looking into this.
>> If it turned out that tracing spans are not reached to
>> actual HDFS writer thread in HBase, I will file a JIRA.
>>
>> # We need hadoop-2.6.0 or higher in order to trace HDFS.
>> # Building hbase from source with {{-Dhadoop-two.version=2.6.0}}
>> # is straight forward way to do this
>> # because the binary release of hbase-1.0.0 bundles hadoop-2.5.1 jars.
>>
>> Masatake
>>
>>
>> On 2/11/15 08:56, Nick Dimiduk wrote:
>>> Hi Joshua,
>>>
>>> In theory there's nothing special for you to do. Just issue your query to
>>> HBase with tracing enabled. The active span will go through HBase, down
>>> into HDFS, and back again. You'll need both systems collecting spans into
>>> the same place so that you can report on the complete trace tree.
>>>
>>> I've not recently tested the end-to-end, but I believe it's all there. If
>>> not, it's a bug -- this is an intended use case. Can you give it a try
>>> and let us know how it goes?
>>>
>>> FYI, 0.99.x are preview releases of HBase and not for production use. Just
>>> so you know :)
>>>
>>> -n
>>>
>>> On Wednesday, February 11, 2015, Chunxu Tang <ch...@gmail.com> wrote:
>>>
>>>> Hi all,
>>>>
>>>> Now I’m exploiting HTrace to trace request level data flows in HBase and
>>>> HDFS. I have successfully traced HBase and HDFS by using HTrace,
>>>> respectively.
>>>>
>>>> After that, I combine HBase and HDFS together and I want to just send a
>>>> PUT/GET request to HBase, but to trace the whole data flow in both HBase
>>>> and HDFS. In my opinion, when I send a request such as Get to HBase, it
>>>> will at last try to read the blocks on HDFS, so I can construct a whole
>>>> data flow tracing through HBase and HDFS. While, the fact is that I can
>>>> only get tracing data of HBase, with no data of HDFS.
>>>>
>>>> Could you give me any suggestions on how to trace the data flow in both
>>>> HBase and HDFS? Does anyone have similar experience? Do I need to modify
>>>> the source code? And maybe which part(s) should I touch? If I need to
>>>> modify the code, I will try to create a patch for that.
>>>>
>>>> Thank you.
>>>>
>>>> My Configurations:
>>>> Hadoop version: 2.6.0
>>>> HBase version: 0.99.2
>>>> HTrace version: htrace-master
>>>> OS: Ubuntu 12.04
>>>>
>>>>
>>>> Joshua
>>>>


Re: Trace HBase/HDFS with HTrace

Posted by "Colin P. McCabe" <cm...@apache.org>.
Thanks for trying this, Mastake.  I've got HDFS working on my cluster
with tracing and LocalFileSpanReceiver.  Did you try using HBase +
HDFS with LocalFileSpanReceiver?  Be sure to use a build including
HTRACE-112 since LFSR was kind of busted prior to that.

I'm going to do a longer writeup about getting HDFS + HBase working
with other span receivers just as soon as I finish stomping a few more
bugs.

best,
Colin

On Tue, Feb 24, 2015 at 12:04 PM, Masatake Iwasaki
<iw...@oss.nttdata.co.jp> wrote:
> Hi,
>
> Thanks for trying this. I am sorry for late reply.
>
> I tried this today
> by hbase-1.0.1-SANPSHOT built with {{-Dhadoop-two.version=2.7.0-SNAPSHOT}}
> in pseudo distributed cluster
> but failed to get end-to-end trace.
>
> I checked that
> * tracing works for both of hbase and hdfs,
> * hbase runs with 2.7.0-SNAPSHOT jar of hadoop.
>
> When I did do put with tracing on,
> I saw span named "FSHLog.sync" with annotations such as
> "syncing writer" and "writer synced".
> The code for tracing in FSHLog worked at least.
>
> I'm still looking into this.
> If it turned out that tracing spans are not reached to
> actual HDFS writer thread in HBase, I will file a JIRA.
>
> # We need hadoop-2.6.0 or higher in order to trace HDFS.
> # Building hbase from source with {{-Dhadoop-two.version=2.6.0}}
> # is straight forward way to do this
> # because the binary release of hbase-1.0.0 bundles hadoop-2.5.1 jars.
>
> Masatake
>
>
> On 2/11/15 08:56, Nick Dimiduk wrote:
>>
>> Hi Joshua,
>>
>> In theory there's nothing special for you to do. Just issue your query to
>> HBase with tracing enabled. The active span will go through HBase, down
>> into HDFS, and back again. You'll need both systems collecting spans into
>> the same place so that you can report on the complete trace tree.
>>
>> I've not recently tested the end-to-end, but I believe it's all there. If
>> not, it's a bug -- this is an intended use case. Can you give it a try
>> and let us know how it goes?
>>
>> FYI, 0.99.x are preview releases of HBase and not for production use. Just
>> so you know :)
>>
>> -n
>>
>> On Wednesday, February 11, 2015, Chunxu Tang <ch...@gmail.com> wrote:
>>
>>> Hi all,
>>>
>>> Now I’m exploiting HTrace to trace request level data flows in HBase and
>>> HDFS. I have successfully traced HBase and HDFS by using HTrace,
>>> respectively.
>>>
>>> After that, I combine HBase and HDFS together and I want to just send a
>>> PUT/GET request to HBase, but to trace the whole data flow in both HBase
>>> and HDFS. In my opinion, when I send a request such as Get to HBase, it
>>> will at last try to read the blocks on HDFS, so I can construct a whole
>>> data flow tracing through HBase and HDFS. While, the fact is that I can
>>> only get tracing data of HBase, with no data of HDFS.
>>>
>>> Could you give me any suggestions on how to trace the data flow in both
>>> HBase and HDFS? Does anyone have similar experience? Do I need to modify
>>> the source code? And maybe which part(s) should I touch? If I need to
>>> modify the code, I will try to create a patch for that.
>>>
>>> Thank you.
>>>
>>> My Configurations:
>>> Hadoop version: 2.6.0
>>> HBase version: 0.99.2
>>> HTrace version: htrace-master
>>> OS: Ubuntu 12.04
>>>
>>>
>>> Joshua
>>>
>

Re: Trace HBase/HDFS with HTrace

Posted by Masatake Iwasaki <iw...@oss.nttdata.co.jp>.
Hi,

Thanks for trying this. I am sorry for late reply.

I tried this today
by hbase-1.0.1-SANPSHOT built with {{-Dhadoop-two.version=2.7.0-SNAPSHOT}}
in pseudo distributed cluster
but failed to get end-to-end trace.

I checked that
* tracing works for both of hbase and hdfs,
* hbase runs with 2.7.0-SNAPSHOT jar of hadoop.

When I did do put with tracing on,
I saw span named "FSHLog.sync" with annotations such as
"syncing writer" and "writer synced".
The code for tracing in FSHLog worked at least.

I'm still looking into this.
If it turned out that tracing spans are not reached to
actual HDFS writer thread in HBase, I will file a JIRA.

# We need hadoop-2.6.0 or higher in order to trace HDFS.
# Building hbase from source with {{-Dhadoop-two.version=2.6.0}}
# is straight forward way to do this
# because the binary release of hbase-1.0.0 bundles hadoop-2.5.1 jars.

Masatake

On 2/11/15 08:56, Nick Dimiduk wrote:
> Hi Joshua,
>
> In theory there's nothing special for you to do. Just issue your query to
> HBase with tracing enabled. The active span will go through HBase, down
> into HDFS, and back again. You'll need both systems collecting spans into
> the same place so that you can report on the complete trace tree.
>
> I've not recently tested the end-to-end, but I believe it's all there. If
> not, it's a bug -- this is an intended use case. Can you give it a try
> and let us know how it goes?
>
> FYI, 0.99.x are preview releases of HBase and not for production use. Just
> so you know :)
>
> -n
>
> On Wednesday, February 11, 2015, Chunxu Tang <ch...@gmail.com> wrote:
>
>> Hi all,
>>
>> Now I’m exploiting HTrace to trace request level data flows in HBase and
>> HDFS. I have successfully traced HBase and HDFS by using HTrace,
>> respectively.
>>
>> After that, I combine HBase and HDFS together and I want to just send a
>> PUT/GET request to HBase, but to trace the whole data flow in both HBase
>> and HDFS. In my opinion, when I send a request such as Get to HBase, it
>> will at last try to read the blocks on HDFS, so I can construct a whole
>> data flow tracing through HBase and HDFS. While, the fact is that I can
>> only get tracing data of HBase, with no data of HDFS.
>>
>> Could you give me any suggestions on how to trace the data flow in both
>> HBase and HDFS? Does anyone have similar experience? Do I need to modify
>> the source code? And maybe which part(s) should I touch? If I need to
>> modify the code, I will try to create a patch for that.
>>
>> Thank you.
>>
>> My Configurations:
>> Hadoop version: 2.6.0
>> HBase version: 0.99.2
>> HTrace version: htrace-master
>> OS: Ubuntu 12.04
>>
>>
>> Joshua
>>


Re: Trace HBase/HDFS with HTrace

Posted by Nick Dimiduk <nd...@gmail.com>.
Hi Joshua,

In theory there's nothing special for you to do. Just issue your query to
HBase with tracing enabled. The active span will go through HBase, down
into HDFS, and back again. You'll need both systems collecting spans into
the same place so that you can report on the complete trace tree.

I've not recently tested the end-to-end, but I believe it's all there. If
not, it's a bug -- this is an intended use case. Can you give it a try
and let us know how it goes?

FYI, 0.99.x are preview releases of HBase and not for production use. Just
so you know :)

-n

On Wednesday, February 11, 2015, Chunxu Tang <ch...@gmail.com> wrote:

> Hi all,
>
> Now I’m exploiting HTrace to trace request level data flows in HBase and
> HDFS. I have successfully traced HBase and HDFS by using HTrace,
> respectively.
>
> After that, I combine HBase and HDFS together and I want to just send a
> PUT/GET request to HBase, but to trace the whole data flow in both HBase
> and HDFS. In my opinion, when I send a request such as Get to HBase, it
> will at last try to read the blocks on HDFS, so I can construct a whole
> data flow tracing through HBase and HDFS. While, the fact is that I can
> only get tracing data of HBase, with no data of HDFS.
>
> Could you give me any suggestions on how to trace the data flow in both
> HBase and HDFS? Does anyone have similar experience? Do I need to modify
> the source code? And maybe which part(s) should I touch? If I need to
> modify the code, I will try to create a patch for that.
>
> Thank you.
>
> My Configurations:
> Hadoop version: 2.6.0
> HBase version: 0.99.2
> HTrace version: htrace-master
> OS: Ubuntu 12.04
>
>
> Joshua
>

Re: Trace HBase/HDFS with HTrace

Posted by "Colin P. McCabe" <cm...@apache.org>.
On Thu, Feb 12, 2015 at 1:23 PM, Chunxu Tang <ch...@gmail.com> wrote:
> Hi all,
>
> Thanks for your detailed replies!
>
> Now I have tested end-to-end tracing in two versions of HBase (0.98.10 and
> 0.99.2), combined with Hadoop 2.6.0 and htrace-master (3.0.4), and both of
> them failed. For HBase 0.98.10, it actually has htrace 2.0.4 core, so it's
> normal to get no traces. While, HBase 0.99.2 has htrace 3.0.4 core, but I
> still cannot get traces of HDFS, I can only get traces of HBase.

Hadoop 2.6.0 doesn't the correct version of HTrace, so this is all
expected.  You aren't going to be able to do anything useful here as
long as you keep using Hadoop 2.6.0.  I would suggest using
Hadoop-2.7.0-SNAPSHOT with an appropriate version of HBase.

Hope this helps.

best,
Colin


>
> I think the first thing I need to make sure is that I use a correct method
> to implement end-to-end test. I'm not very sure whether it's good to show
> whole source code on the mailing list, so I just put some core code chunks
> written in the client code here:
>
> public void run(){
>         Configuration conf = HBaseConfiguration.create();
>         org.apache.hadoop.hbase.trace.SpanReceiverHost.getInstance(conf);
>         org.apache.hadoop.tracing.SpanReceiverHost.getInstance(new
> HdfsConfiguration());
>
>         TraceScope ts = Trace.startSpan("Gets", Sampler.ALWAYS);
>         HTable table = new HTable(conf, "t1");
>         Get get = new Get(Bytes.toBytes("r1"));
>         table.get(get);
>         ...
> }
>
> Now I can only get traces of HBase, ending with HfileReaderV2.readBlock()
> function. Is my testing method correct? And because I'm not familiar with
> new version of HTrace and HBase/HDFS with new htrace core, could you give
> me some suggestions to detect where the error may take place?
>
> Thank you all.
>
> Joshua
>
> 2015-02-11 22:08 GMT-05:00 Colin P. McCabe <cm...@apache.org>:
>
>> No, I think I'm the one who's missing something. :)
>>
>> I will give that a try next time I'm testing out end-to-end tracing.
>>
>> thanks guys.
>> Colin
>>
>> On Wed, Feb 11, 2015 at 4:36 PM, Enis Söztutar <en...@gmail.com> wrote:
>> > mvn install just installs it in local cache which you can then use for
>> > building other projects. So no need to have to define a file based local
>> > repo. Am I missing something?
>> >
>> > Enis
>> >
>> > On Wed, Feb 11, 2015 at 12:36 PM, Nick Dimiduk <nd...@gmail.com>
>> wrote:
>> >
>> >> Oh, I see. I was assuming a local build of Hadoop snapshot installed
>> into
>> >> the local cache.
>> >>
>> >> On Wednesday, February 11, 2015, Colin P. McCabe <cm...@apache.org>
>> >> wrote:
>> >>
>> >> > On Wed, Feb 11, 2015 at 11:27 AM, Nick Dimiduk <ndimiduk@gmail.com
>> >> > <javascript:;>> wrote:
>> >> > > I don't recall the hadoop release repo restriction being a problem,
>> >> but I
>> >> > > haven't tested it lately. See if you can just specify the release
>> >> version
>> >> > > with -Dhadoop.version or -Dhadoop-two.version.
>> >> > >
>> >> >
>> >> > Sorry, it's been a while since I did this... I guess the question is
>> >> > whether 2.7.0-SNAPSHOT is available in Maven-land somewhere?  If so,
>> >> > then Chunxu should forget all that stuff I said, and just build HBase
>> >> > with -Dhadoop.version=2.7.0-SNAPSHOT
>> >> >
>> >> > > I would go against branch-1.0 as this will be the eminent 1.0.0
>> release
>> >> > and
>> >> > > had HTrace 3.1.0-incubating.
>> >> >
>> >> > Thanks.
>> >> >
>> >> > Colin
>> >> >
>> >> >
>> >> > >
>> >> > > -n
>> >> > >
>> >> > > On Wed, Feb 11, 2015 at 11:13 AM, Colin P. McCabe <
>> cmccabe@apache.org
>> >> > <javascript:;>>
>> >> > > wrote:
>> >> > >
>> >> > >> Thanks for trying stuff out!  Sorry that this is a little
>> difficult at
>> >> > >> the moment.
>> >> > >>
>> >> > >> To really do this right, you would want to be using Hadoop with
>> HTrace
>> >> > >> 3.1.0, and HBase with HTrace 3.1.0.  Unfortunately, there hasn't
>> been
>> >> > >> a new release of Hadoop with HTrace 3.1.0.  The only existing
>> releases
>> >> > >> of Hadoop use an older version of the HTrace library.  So you will
>> >> > >> have to build from source.
>> >> > >>
>> >> > >> If you check out Hadoop's "branch-2" branch (currently, this branch
>> >> > >> represents what will be in the 2.7 release, when it is cut), and
>> build
>> >> > >> that, you will get the latest.  Then you have to build a version of
>> >> > >> HBase against the version of Hadoop you have built.
>> >> > >>
>> >> > >> By default, HBase's Maven build will build against upstream release
>> >> > >> versions of Hadoop only. So just setting
>> >> > >> -Dhadoop.version=2.7.0-SNAPSHOT is not enough, since it won't know
>> >> > >> where to find the jars.  To get around this problem, you can create
>> >> > >> your own local maven repo. Here's how.
>> >> > >>
>> >> > >> In hadoop/pom.xml, add these lines to the distributionManagement
>> >> stanza:
>> >> > >>
>> >> > >> +    <repository>
>> >> > >> +      <id>localdump</id>
>> >> > >> +      <url>file:///home/cmccabe/localdump/releases</url>
>> >> > >> +    </repository>
>> >> > >> +    <snapshotRepository>
>> >> > >> +      <id>localdump</id>
>> >> > >> +      <url>file:///home/cmccabe/localdump/snapshots</url>
>> >> > >> +    </snapshotRepository>
>> >> > >>
>> >> > >> Comment out the repositories that are already there.
>> >> > >>
>> >> > >> Now run mkdir /home/cmccabe/localdump.
>> >> > >>
>> >> > >> Then, in your hadoop tree, run mvn deploy -DskipTests.
>> >> > >>
>> >> > >> You should get a localdump directory that has files kind of like
>> this:
>> >> > >>
>> >> > >> ...
>> >> > >> /home/cmccabe/localdump/snapshots/org/apache/hadoop
>> >> > >>
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce
>> >> > >>
>> >> > >>
>> >> >
>> >>
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/maven-metadata.xml.md5
>> >> > >>
>> >> > >>
>> >> >
>> >>
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT
>> >> > >>
>> >> > >>
>> >> >
>> >>
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml.md5
>> >> > >>
>> >> > >>
>> >> >
>> >>
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/hadoop-mapreduce-2.7.0-20121120.230341-1.pom.sha1
>> >> > >>
>> >> > >>
>> >> >
>> >>
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml
>> >> > >> ...
>> >> > >>
>> >> > >> Now, add the following lines to your HBase pom.xml:
>> >> > >>
>> >> > >>    <repositories>
>> >> > >>      <repository>
>> >> > >> +      <id>localdump</id>
>> >> > >> +      <url>file:///home/cmccabe/localdump</url>
>> >> > >> +      <name>Local Dump</name>
>> >> > >> +      <snapshots>
>> >> > >> +        <enabled>true</enabled>
>> >> > >> +      </snapshots>
>> >> > >> +      <releases>
>> >> > >> +        <enabled>true</enabled>
>> >> > >> +      </releases>
>> >> > >> +    </repository>
>> >> > >> +    <repository>
>> >> > >>
>> >> > >> This will allow you to run something like:
>> >> > >> mvn test -Dtest=TestMiniClusterLoadSequential -PlocalTests
>> >> > >> -DredirectTestOutputToFile=true -Dhadoop.profile=2.0
>> >> > >> -Dhadoop.version=2.7.0-SNAPSHOT -Dcdh.hadoop.version=2.7.0-SNAPSHOT
>> >> > >>
>> >> > >> Once we do a new release of Hadoop with HTrace 3.1.0 this will get
>> a
>> >> lot
>> >> > >> easier.
>> >> > >>
>> >> > >> Related: Does anyone know what the best git branch to build from
>> for
>> >> > >> HBase would be for this kind of testing?  I've been meaning to do
>> some
>> >> > >> end to end testing (it's been on my TODO for a while)
>> >> > >>
>> >> > >> best,
>> >> > >> Colin
>> >> > >>
>> >> > >> On Wed, Feb 11, 2015 at 7:55 AM, Chunxu Tang <chunxutang@gmail.com
>> >> > <javascript:;>> wrote:
>> >> > >> > Hi all,
>> >> > >> >
>> >> > >> > Now I’m exploiting HTrace to trace request level data flows in
>> HBase
>> >> > and
>> >> > >> > HDFS. I have successfully traced HBase and HDFS by using HTrace,
>> >> > >> > respectively.
>> >> > >> >
>> >> > >> > After that, I combine HBase and HDFS together and I want to just
>> >> send
>> >> > a
>> >> > >> > PUT/GET request to HBase, but to trace the whole data flow in
>> both
>> >> > HBase
>> >> > >> > and HDFS. In my opinion, when I send a request such as Get to
>> HBase,
>> >> > it
>> >> > >> > will at last try to read the blocks on HDFS, so I can construct a
>> >> > whole
>> >> > >> > data flow tracing through HBase and HDFS. While, the fact is
>> that I
>> >> > can
>> >> > >> > only get tracing data of HBase, with no data of HDFS.
>> >> > >> >
>> >> > >> > Could you give me any suggestions on how to trace the data flow
>> in
>> >> > both
>> >> > >> > HBase and HDFS? Does anyone have similar experience? Do I need to
>> >> > modify
>> >> > >> > the source code? And maybe which part(s) should I touch? If I
>> need
>> >> to
>> >> > >> > modify the code, I will try to create a patch for that.
>> >> > >> >
>> >> > >> > Thank you.
>> >> > >> >
>> >> > >> > My Configurations:
>> >> > >> > Hadoop version: 2.6.0
>> >> > >> > HBase version: 0.99.2
>> >> > >> > HTrace version: htrace-master
>> >> > >> > OS: Ubuntu 12.04
>> >> > >> >
>> >> > >> >
>> >> > >> > Joshua
>> >> > >>
>> >> >
>> >>
>>

Re: Trace HBase/HDFS with HTrace

Posted by Chunxu Tang <ch...@gmail.com>.
Hi all,

Thanks for your detailed replies!

Now I have tested end-to-end tracing in two versions of HBase (0.98.10 and
0.99.2), combined with Hadoop 2.6.0 and htrace-master (3.0.4), and both of
them failed. For HBase 0.98.10, it actually has htrace 2.0.4 core, so it's
normal to get no traces. While, HBase 0.99.2 has htrace 3.0.4 core, but I
still cannot get traces of HDFS, I can only get traces of HBase.

I think the first thing I need to make sure is that I use a correct method
to implement end-to-end test. I'm not very sure whether it's good to show
whole source code on the mailing list, so I just put some core code chunks
written in the client code here:

public void run(){
        Configuration conf = HBaseConfiguration.create();
        org.apache.hadoop.hbase.trace.SpanReceiverHost.getInstance(conf);
        org.apache.hadoop.tracing.SpanReceiverHost.getInstance(new
HdfsConfiguration());

        TraceScope ts = Trace.startSpan("Gets", Sampler.ALWAYS);
        HTable table = new HTable(conf, "t1");
        Get get = new Get(Bytes.toBytes("r1"));
        table.get(get);
        ...
}

Now I can only get traces of HBase, ending with HfileReaderV2.readBlock()
function. Is my testing method correct? And because I'm not familiar with
new version of HTrace and HBase/HDFS with new htrace core, could you give
me some suggestions to detect where the error may take place?

Thank you all.

Joshua

2015-02-11 22:08 GMT-05:00 Colin P. McCabe <cm...@apache.org>:

> No, I think I'm the one who's missing something. :)
>
> I will give that a try next time I'm testing out end-to-end tracing.
>
> thanks guys.
> Colin
>
> On Wed, Feb 11, 2015 at 4:36 PM, Enis Söztutar <en...@gmail.com> wrote:
> > mvn install just installs it in local cache which you can then use for
> > building other projects. So no need to have to define a file based local
> > repo. Am I missing something?
> >
> > Enis
> >
> > On Wed, Feb 11, 2015 at 12:36 PM, Nick Dimiduk <nd...@gmail.com>
> wrote:
> >
> >> Oh, I see. I was assuming a local build of Hadoop snapshot installed
> into
> >> the local cache.
> >>
> >> On Wednesday, February 11, 2015, Colin P. McCabe <cm...@apache.org>
> >> wrote:
> >>
> >> > On Wed, Feb 11, 2015 at 11:27 AM, Nick Dimiduk <ndimiduk@gmail.com
> >> > <javascript:;>> wrote:
> >> > > I don't recall the hadoop release repo restriction being a problem,
> >> but I
> >> > > haven't tested it lately. See if you can just specify the release
> >> version
> >> > > with -Dhadoop.version or -Dhadoop-two.version.
> >> > >
> >> >
> >> > Sorry, it's been a while since I did this... I guess the question is
> >> > whether 2.7.0-SNAPSHOT is available in Maven-land somewhere?  If so,
> >> > then Chunxu should forget all that stuff I said, and just build HBase
> >> > with -Dhadoop.version=2.7.0-SNAPSHOT
> >> >
> >> > > I would go against branch-1.0 as this will be the eminent 1.0.0
> release
> >> > and
> >> > > had HTrace 3.1.0-incubating.
> >> >
> >> > Thanks.
> >> >
> >> > Colin
> >> >
> >> >
> >> > >
> >> > > -n
> >> > >
> >> > > On Wed, Feb 11, 2015 at 11:13 AM, Colin P. McCabe <
> cmccabe@apache.org
> >> > <javascript:;>>
> >> > > wrote:
> >> > >
> >> > >> Thanks for trying stuff out!  Sorry that this is a little
> difficult at
> >> > >> the moment.
> >> > >>
> >> > >> To really do this right, you would want to be using Hadoop with
> HTrace
> >> > >> 3.1.0, and HBase with HTrace 3.1.0.  Unfortunately, there hasn't
> been
> >> > >> a new release of Hadoop with HTrace 3.1.0.  The only existing
> releases
> >> > >> of Hadoop use an older version of the HTrace library.  So you will
> >> > >> have to build from source.
> >> > >>
> >> > >> If you check out Hadoop's "branch-2" branch (currently, this branch
> >> > >> represents what will be in the 2.7 release, when it is cut), and
> build
> >> > >> that, you will get the latest.  Then you have to build a version of
> >> > >> HBase against the version of Hadoop you have built.
> >> > >>
> >> > >> By default, HBase's Maven build will build against upstream release
> >> > >> versions of Hadoop only. So just setting
> >> > >> -Dhadoop.version=2.7.0-SNAPSHOT is not enough, since it won't know
> >> > >> where to find the jars.  To get around this problem, you can create
> >> > >> your own local maven repo. Here's how.
> >> > >>
> >> > >> In hadoop/pom.xml, add these lines to the distributionManagement
> >> stanza:
> >> > >>
> >> > >> +    <repository>
> >> > >> +      <id>localdump</id>
> >> > >> +      <url>file:///home/cmccabe/localdump/releases</url>
> >> > >> +    </repository>
> >> > >> +    <snapshotRepository>
> >> > >> +      <id>localdump</id>
> >> > >> +      <url>file:///home/cmccabe/localdump/snapshots</url>
> >> > >> +    </snapshotRepository>
> >> > >>
> >> > >> Comment out the repositories that are already there.
> >> > >>
> >> > >> Now run mkdir /home/cmccabe/localdump.
> >> > >>
> >> > >> Then, in your hadoop tree, run mvn deploy -DskipTests.
> >> > >>
> >> > >> You should get a localdump directory that has files kind of like
> this:
> >> > >>
> >> > >> ...
> >> > >> /home/cmccabe/localdump/snapshots/org/apache/hadoop
> >> > >>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce
> >> > >>
> >> > >>
> >> >
> >>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/maven-metadata.xml.md5
> >> > >>
> >> > >>
> >> >
> >>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT
> >> > >>
> >> > >>
> >> >
> >>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml.md5
> >> > >>
> >> > >>
> >> >
> >>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/hadoop-mapreduce-2.7.0-20121120.230341-1.pom.sha1
> >> > >>
> >> > >>
> >> >
> >>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml
> >> > >> ...
> >> > >>
> >> > >> Now, add the following lines to your HBase pom.xml:
> >> > >>
> >> > >>    <repositories>
> >> > >>      <repository>
> >> > >> +      <id>localdump</id>
> >> > >> +      <url>file:///home/cmccabe/localdump</url>
> >> > >> +      <name>Local Dump</name>
> >> > >> +      <snapshots>
> >> > >> +        <enabled>true</enabled>
> >> > >> +      </snapshots>
> >> > >> +      <releases>
> >> > >> +        <enabled>true</enabled>
> >> > >> +      </releases>
> >> > >> +    </repository>
> >> > >> +    <repository>
> >> > >>
> >> > >> This will allow you to run something like:
> >> > >> mvn test -Dtest=TestMiniClusterLoadSequential -PlocalTests
> >> > >> -DredirectTestOutputToFile=true -Dhadoop.profile=2.0
> >> > >> -Dhadoop.version=2.7.0-SNAPSHOT -Dcdh.hadoop.version=2.7.0-SNAPSHOT
> >> > >>
> >> > >> Once we do a new release of Hadoop with HTrace 3.1.0 this will get
> a
> >> lot
> >> > >> easier.
> >> > >>
> >> > >> Related: Does anyone know what the best git branch to build from
> for
> >> > >> HBase would be for this kind of testing?  I've been meaning to do
> some
> >> > >> end to end testing (it's been on my TODO for a while)
> >> > >>
> >> > >> best,
> >> > >> Colin
> >> > >>
> >> > >> On Wed, Feb 11, 2015 at 7:55 AM, Chunxu Tang <chunxutang@gmail.com
> >> > <javascript:;>> wrote:
> >> > >> > Hi all,
> >> > >> >
> >> > >> > Now I’m exploiting HTrace to trace request level data flows in
> HBase
> >> > and
> >> > >> > HDFS. I have successfully traced HBase and HDFS by using HTrace,
> >> > >> > respectively.
> >> > >> >
> >> > >> > After that, I combine HBase and HDFS together and I want to just
> >> send
> >> > a
> >> > >> > PUT/GET request to HBase, but to trace the whole data flow in
> both
> >> > HBase
> >> > >> > and HDFS. In my opinion, when I send a request such as Get to
> HBase,
> >> > it
> >> > >> > will at last try to read the blocks on HDFS, so I can construct a
> >> > whole
> >> > >> > data flow tracing through HBase and HDFS. While, the fact is
> that I
> >> > can
> >> > >> > only get tracing data of HBase, with no data of HDFS.
> >> > >> >
> >> > >> > Could you give me any suggestions on how to trace the data flow
> in
> >> > both
> >> > >> > HBase and HDFS? Does anyone have similar experience? Do I need to
> >> > modify
> >> > >> > the source code? And maybe which part(s) should I touch? If I
> need
> >> to
> >> > >> > modify the code, I will try to create a patch for that.
> >> > >> >
> >> > >> > Thank you.
> >> > >> >
> >> > >> > My Configurations:
> >> > >> > Hadoop version: 2.6.0
> >> > >> > HBase version: 0.99.2
> >> > >> > HTrace version: htrace-master
> >> > >> > OS: Ubuntu 12.04
> >> > >> >
> >> > >> >
> >> > >> > Joshua
> >> > >>
> >> >
> >>
>

Re: Trace HBase/HDFS with HTrace

Posted by "Colin P. McCabe" <cm...@apache.org>.
No, I think I'm the one who's missing something. :)

I will give that a try next time I'm testing out end-to-end tracing.

thanks guys.
Colin

On Wed, Feb 11, 2015 at 4:36 PM, Enis Söztutar <en...@gmail.com> wrote:
> mvn install just installs it in local cache which you can then use for
> building other projects. So no need to have to define a file based local
> repo. Am I missing something?
>
> Enis
>
> On Wed, Feb 11, 2015 at 12:36 PM, Nick Dimiduk <nd...@gmail.com> wrote:
>
>> Oh, I see. I was assuming a local build of Hadoop snapshot installed into
>> the local cache.
>>
>> On Wednesday, February 11, 2015, Colin P. McCabe <cm...@apache.org>
>> wrote:
>>
>> > On Wed, Feb 11, 2015 at 11:27 AM, Nick Dimiduk <ndimiduk@gmail.com
>> > <javascript:;>> wrote:
>> > > I don't recall the hadoop release repo restriction being a problem,
>> but I
>> > > haven't tested it lately. See if you can just specify the release
>> version
>> > > with -Dhadoop.version or -Dhadoop-two.version.
>> > >
>> >
>> > Sorry, it's been a while since I did this... I guess the question is
>> > whether 2.7.0-SNAPSHOT is available in Maven-land somewhere?  If so,
>> > then Chunxu should forget all that stuff I said, and just build HBase
>> > with -Dhadoop.version=2.7.0-SNAPSHOT
>> >
>> > > I would go against branch-1.0 as this will be the eminent 1.0.0 release
>> > and
>> > > had HTrace 3.1.0-incubating.
>> >
>> > Thanks.
>> >
>> > Colin
>> >
>> >
>> > >
>> > > -n
>> > >
>> > > On Wed, Feb 11, 2015 at 11:13 AM, Colin P. McCabe <cmccabe@apache.org
>> > <javascript:;>>
>> > > wrote:
>> > >
>> > >> Thanks for trying stuff out!  Sorry that this is a little difficult at
>> > >> the moment.
>> > >>
>> > >> To really do this right, you would want to be using Hadoop with HTrace
>> > >> 3.1.0, and HBase with HTrace 3.1.0.  Unfortunately, there hasn't been
>> > >> a new release of Hadoop with HTrace 3.1.0.  The only existing releases
>> > >> of Hadoop use an older version of the HTrace library.  So you will
>> > >> have to build from source.
>> > >>
>> > >> If you check out Hadoop's "branch-2" branch (currently, this branch
>> > >> represents what will be in the 2.7 release, when it is cut), and build
>> > >> that, you will get the latest.  Then you have to build a version of
>> > >> HBase against the version of Hadoop you have built.
>> > >>
>> > >> By default, HBase's Maven build will build against upstream release
>> > >> versions of Hadoop only. So just setting
>> > >> -Dhadoop.version=2.7.0-SNAPSHOT is not enough, since it won't know
>> > >> where to find the jars.  To get around this problem, you can create
>> > >> your own local maven repo. Here's how.
>> > >>
>> > >> In hadoop/pom.xml, add these lines to the distributionManagement
>> stanza:
>> > >>
>> > >> +    <repository>
>> > >> +      <id>localdump</id>
>> > >> +      <url>file:///home/cmccabe/localdump/releases</url>
>> > >> +    </repository>
>> > >> +    <snapshotRepository>
>> > >> +      <id>localdump</id>
>> > >> +      <url>file:///home/cmccabe/localdump/snapshots</url>
>> > >> +    </snapshotRepository>
>> > >>
>> > >> Comment out the repositories that are already there.
>> > >>
>> > >> Now run mkdir /home/cmccabe/localdump.
>> > >>
>> > >> Then, in your hadoop tree, run mvn deploy -DskipTests.
>> > >>
>> > >> You should get a localdump directory that has files kind of like this:
>> > >>
>> > >> ...
>> > >> /home/cmccabe/localdump/snapshots/org/apache/hadoop
>> > >> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce
>> > >>
>> > >>
>> >
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/maven-metadata.xml.md5
>> > >>
>> > >>
>> >
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT
>> > >>
>> > >>
>> >
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml.md5
>> > >>
>> > >>
>> >
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/hadoop-mapreduce-2.7.0-20121120.230341-1.pom.sha1
>> > >>
>> > >>
>> >
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml
>> > >> ...
>> > >>
>> > >> Now, add the following lines to your HBase pom.xml:
>> > >>
>> > >>    <repositories>
>> > >>      <repository>
>> > >> +      <id>localdump</id>
>> > >> +      <url>file:///home/cmccabe/localdump</url>
>> > >> +      <name>Local Dump</name>
>> > >> +      <snapshots>
>> > >> +        <enabled>true</enabled>
>> > >> +      </snapshots>
>> > >> +      <releases>
>> > >> +        <enabled>true</enabled>
>> > >> +      </releases>
>> > >> +    </repository>
>> > >> +    <repository>
>> > >>
>> > >> This will allow you to run something like:
>> > >> mvn test -Dtest=TestMiniClusterLoadSequential -PlocalTests
>> > >> -DredirectTestOutputToFile=true -Dhadoop.profile=2.0
>> > >> -Dhadoop.version=2.7.0-SNAPSHOT -Dcdh.hadoop.version=2.7.0-SNAPSHOT
>> > >>
>> > >> Once we do a new release of Hadoop with HTrace 3.1.0 this will get a
>> lot
>> > >> easier.
>> > >>
>> > >> Related: Does anyone know what the best git branch to build from for
>> > >> HBase would be for this kind of testing?  I've been meaning to do some
>> > >> end to end testing (it's been on my TODO for a while)
>> > >>
>> > >> best,
>> > >> Colin
>> > >>
>> > >> On Wed, Feb 11, 2015 at 7:55 AM, Chunxu Tang <chunxutang@gmail.com
>> > <javascript:;>> wrote:
>> > >> > Hi all,
>> > >> >
>> > >> > Now I’m exploiting HTrace to trace request level data flows in HBase
>> > and
>> > >> > HDFS. I have successfully traced HBase and HDFS by using HTrace,
>> > >> > respectively.
>> > >> >
>> > >> > After that, I combine HBase and HDFS together and I want to just
>> send
>> > a
>> > >> > PUT/GET request to HBase, but to trace the whole data flow in both
>> > HBase
>> > >> > and HDFS. In my opinion, when I send a request such as Get to HBase,
>> > it
>> > >> > will at last try to read the blocks on HDFS, so I can construct a
>> > whole
>> > >> > data flow tracing through HBase and HDFS. While, the fact is that I
>> > can
>> > >> > only get tracing data of HBase, with no data of HDFS.
>> > >> >
>> > >> > Could you give me any suggestions on how to trace the data flow in
>> > both
>> > >> > HBase and HDFS? Does anyone have similar experience? Do I need to
>> > modify
>> > >> > the source code? And maybe which part(s) should I touch? If I need
>> to
>> > >> > modify the code, I will try to create a patch for that.
>> > >> >
>> > >> > Thank you.
>> > >> >
>> > >> > My Configurations:
>> > >> > Hadoop version: 2.6.0
>> > >> > HBase version: 0.99.2
>> > >> > HTrace version: htrace-master
>> > >> > OS: Ubuntu 12.04
>> > >> >
>> > >> >
>> > >> > Joshua
>> > >>
>> >
>>

Re: Trace HBase/HDFS with HTrace

Posted by Enis Söztutar <en...@gmail.com>.
mvn install just installs it in local cache which you can then use for
building other projects. So no need to have to define a file based local
repo. Am I missing something?

Enis

On Wed, Feb 11, 2015 at 12:36 PM, Nick Dimiduk <nd...@gmail.com> wrote:

> Oh, I see. I was assuming a local build of Hadoop snapshot installed into
> the local cache.
>
> On Wednesday, February 11, 2015, Colin P. McCabe <cm...@apache.org>
> wrote:
>
> > On Wed, Feb 11, 2015 at 11:27 AM, Nick Dimiduk <ndimiduk@gmail.com
> > <javascript:;>> wrote:
> > > I don't recall the hadoop release repo restriction being a problem,
> but I
> > > haven't tested it lately. See if you can just specify the release
> version
> > > with -Dhadoop.version or -Dhadoop-two.version.
> > >
> >
> > Sorry, it's been a while since I did this... I guess the question is
> > whether 2.7.0-SNAPSHOT is available in Maven-land somewhere?  If so,
> > then Chunxu should forget all that stuff I said, and just build HBase
> > with -Dhadoop.version=2.7.0-SNAPSHOT
> >
> > > I would go against branch-1.0 as this will be the eminent 1.0.0 release
> > and
> > > had HTrace 3.1.0-incubating.
> >
> > Thanks.
> >
> > Colin
> >
> >
> > >
> > > -n
> > >
> > > On Wed, Feb 11, 2015 at 11:13 AM, Colin P. McCabe <cmccabe@apache.org
> > <javascript:;>>
> > > wrote:
> > >
> > >> Thanks for trying stuff out!  Sorry that this is a little difficult at
> > >> the moment.
> > >>
> > >> To really do this right, you would want to be using Hadoop with HTrace
> > >> 3.1.0, and HBase with HTrace 3.1.0.  Unfortunately, there hasn't been
> > >> a new release of Hadoop with HTrace 3.1.0.  The only existing releases
> > >> of Hadoop use an older version of the HTrace library.  So you will
> > >> have to build from source.
> > >>
> > >> If you check out Hadoop's "branch-2" branch (currently, this branch
> > >> represents what will be in the 2.7 release, when it is cut), and build
> > >> that, you will get the latest.  Then you have to build a version of
> > >> HBase against the version of Hadoop you have built.
> > >>
> > >> By default, HBase's Maven build will build against upstream release
> > >> versions of Hadoop only. So just setting
> > >> -Dhadoop.version=2.7.0-SNAPSHOT is not enough, since it won't know
> > >> where to find the jars.  To get around this problem, you can create
> > >> your own local maven repo. Here's how.
> > >>
> > >> In hadoop/pom.xml, add these lines to the distributionManagement
> stanza:
> > >>
> > >> +    <repository>
> > >> +      <id>localdump</id>
> > >> +      <url>file:///home/cmccabe/localdump/releases</url>
> > >> +    </repository>
> > >> +    <snapshotRepository>
> > >> +      <id>localdump</id>
> > >> +      <url>file:///home/cmccabe/localdump/snapshots</url>
> > >> +    </snapshotRepository>
> > >>
> > >> Comment out the repositories that are already there.
> > >>
> > >> Now run mkdir /home/cmccabe/localdump.
> > >>
> > >> Then, in your hadoop tree, run mvn deploy -DskipTests.
> > >>
> > >> You should get a localdump directory that has files kind of like this:
> > >>
> > >> ...
> > >> /home/cmccabe/localdump/snapshots/org/apache/hadoop
> > >> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce
> > >>
> > >>
> >
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/maven-metadata.xml.md5
> > >>
> > >>
> >
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT
> > >>
> > >>
> >
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml.md5
> > >>
> > >>
> >
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/hadoop-mapreduce-2.7.0-20121120.230341-1.pom.sha1
> > >>
> > >>
> >
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml
> > >> ...
> > >>
> > >> Now, add the following lines to your HBase pom.xml:
> > >>
> > >>    <repositories>
> > >>      <repository>
> > >> +      <id>localdump</id>
> > >> +      <url>file:///home/cmccabe/localdump</url>
> > >> +      <name>Local Dump</name>
> > >> +      <snapshots>
> > >> +        <enabled>true</enabled>
> > >> +      </snapshots>
> > >> +      <releases>
> > >> +        <enabled>true</enabled>
> > >> +      </releases>
> > >> +    </repository>
> > >> +    <repository>
> > >>
> > >> This will allow you to run something like:
> > >> mvn test -Dtest=TestMiniClusterLoadSequential -PlocalTests
> > >> -DredirectTestOutputToFile=true -Dhadoop.profile=2.0
> > >> -Dhadoop.version=2.7.0-SNAPSHOT -Dcdh.hadoop.version=2.7.0-SNAPSHOT
> > >>
> > >> Once we do a new release of Hadoop with HTrace 3.1.0 this will get a
> lot
> > >> easier.
> > >>
> > >> Related: Does anyone know what the best git branch to build from for
> > >> HBase would be for this kind of testing?  I've been meaning to do some
> > >> end to end testing (it's been on my TODO for a while)
> > >>
> > >> best,
> > >> Colin
> > >>
> > >> On Wed, Feb 11, 2015 at 7:55 AM, Chunxu Tang <chunxutang@gmail.com
> > <javascript:;>> wrote:
> > >> > Hi all,
> > >> >
> > >> > Now I’m exploiting HTrace to trace request level data flows in HBase
> > and
> > >> > HDFS. I have successfully traced HBase and HDFS by using HTrace,
> > >> > respectively.
> > >> >
> > >> > After that, I combine HBase and HDFS together and I want to just
> send
> > a
> > >> > PUT/GET request to HBase, but to trace the whole data flow in both
> > HBase
> > >> > and HDFS. In my opinion, when I send a request such as Get to HBase,
> > it
> > >> > will at last try to read the blocks on HDFS, so I can construct a
> > whole
> > >> > data flow tracing through HBase and HDFS. While, the fact is that I
> > can
> > >> > only get tracing data of HBase, with no data of HDFS.
> > >> >
> > >> > Could you give me any suggestions on how to trace the data flow in
> > both
> > >> > HBase and HDFS? Does anyone have similar experience? Do I need to
> > modify
> > >> > the source code? And maybe which part(s) should I touch? If I need
> to
> > >> > modify the code, I will try to create a patch for that.
> > >> >
> > >> > Thank you.
> > >> >
> > >> > My Configurations:
> > >> > Hadoop version: 2.6.0
> > >> > HBase version: 0.99.2
> > >> > HTrace version: htrace-master
> > >> > OS: Ubuntu 12.04
> > >> >
> > >> >
> > >> > Joshua
> > >>
> >
>

Re: Trace HBase/HDFS with HTrace

Posted by Nick Dimiduk <nd...@gmail.com>.
Oh, I see. I was assuming a local build of Hadoop snapshot installed into
the local cache.

On Wednesday, February 11, 2015, Colin P. McCabe <cm...@apache.org> wrote:

> On Wed, Feb 11, 2015 at 11:27 AM, Nick Dimiduk <ndimiduk@gmail.com
> <javascript:;>> wrote:
> > I don't recall the hadoop release repo restriction being a problem, but I
> > haven't tested it lately. See if you can just specify the release version
> > with -Dhadoop.version or -Dhadoop-two.version.
> >
>
> Sorry, it's been a while since I did this... I guess the question is
> whether 2.7.0-SNAPSHOT is available in Maven-land somewhere?  If so,
> then Chunxu should forget all that stuff I said, and just build HBase
> with -Dhadoop.version=2.7.0-SNAPSHOT
>
> > I would go against branch-1.0 as this will be the eminent 1.0.0 release
> and
> > had HTrace 3.1.0-incubating.
>
> Thanks.
>
> Colin
>
>
> >
> > -n
> >
> > On Wed, Feb 11, 2015 at 11:13 AM, Colin P. McCabe <cmccabe@apache.org
> <javascript:;>>
> > wrote:
> >
> >> Thanks for trying stuff out!  Sorry that this is a little difficult at
> >> the moment.
> >>
> >> To really do this right, you would want to be using Hadoop with HTrace
> >> 3.1.0, and HBase with HTrace 3.1.0.  Unfortunately, there hasn't been
> >> a new release of Hadoop with HTrace 3.1.0.  The only existing releases
> >> of Hadoop use an older version of the HTrace library.  So you will
> >> have to build from source.
> >>
> >> If you check out Hadoop's "branch-2" branch (currently, this branch
> >> represents what will be in the 2.7 release, when it is cut), and build
> >> that, you will get the latest.  Then you have to build a version of
> >> HBase against the version of Hadoop you have built.
> >>
> >> By default, HBase's Maven build will build against upstream release
> >> versions of Hadoop only. So just setting
> >> -Dhadoop.version=2.7.0-SNAPSHOT is not enough, since it won't know
> >> where to find the jars.  To get around this problem, you can create
> >> your own local maven repo. Here's how.
> >>
> >> In hadoop/pom.xml, add these lines to the distributionManagement stanza:
> >>
> >> +    <repository>
> >> +      <id>localdump</id>
> >> +      <url>file:///home/cmccabe/localdump/releases</url>
> >> +    </repository>
> >> +    <snapshotRepository>
> >> +      <id>localdump</id>
> >> +      <url>file:///home/cmccabe/localdump/snapshots</url>
> >> +    </snapshotRepository>
> >>
> >> Comment out the repositories that are already there.
> >>
> >> Now run mkdir /home/cmccabe/localdump.
> >>
> >> Then, in your hadoop tree, run mvn deploy -DskipTests.
> >>
> >> You should get a localdump directory that has files kind of like this:
> >>
> >> ...
> >> /home/cmccabe/localdump/snapshots/org/apache/hadoop
> >> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce
> >>
> >>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/maven-metadata.xml.md5
> >>
> >>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT
> >>
> >>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml.md5
> >>
> >>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/hadoop-mapreduce-2.7.0-20121120.230341-1.pom.sha1
> >>
> >>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml
> >> ...
> >>
> >> Now, add the following lines to your HBase pom.xml:
> >>
> >>    <repositories>
> >>      <repository>
> >> +      <id>localdump</id>
> >> +      <url>file:///home/cmccabe/localdump</url>
> >> +      <name>Local Dump</name>
> >> +      <snapshots>
> >> +        <enabled>true</enabled>
> >> +      </snapshots>
> >> +      <releases>
> >> +        <enabled>true</enabled>
> >> +      </releases>
> >> +    </repository>
> >> +    <repository>
> >>
> >> This will allow you to run something like:
> >> mvn test -Dtest=TestMiniClusterLoadSequential -PlocalTests
> >> -DredirectTestOutputToFile=true -Dhadoop.profile=2.0
> >> -Dhadoop.version=2.7.0-SNAPSHOT -Dcdh.hadoop.version=2.7.0-SNAPSHOT
> >>
> >> Once we do a new release of Hadoop with HTrace 3.1.0 this will get a lot
> >> easier.
> >>
> >> Related: Does anyone know what the best git branch to build from for
> >> HBase would be for this kind of testing?  I've been meaning to do some
> >> end to end testing (it's been on my TODO for a while)
> >>
> >> best,
> >> Colin
> >>
> >> On Wed, Feb 11, 2015 at 7:55 AM, Chunxu Tang <chunxutang@gmail.com
> <javascript:;>> wrote:
> >> > Hi all,
> >> >
> >> > Now I’m exploiting HTrace to trace request level data flows in HBase
> and
> >> > HDFS. I have successfully traced HBase and HDFS by using HTrace,
> >> > respectively.
> >> >
> >> > After that, I combine HBase and HDFS together and I want to just send
> a
> >> > PUT/GET request to HBase, but to trace the whole data flow in both
> HBase
> >> > and HDFS. In my opinion, when I send a request such as Get to HBase,
> it
> >> > will at last try to read the blocks on HDFS, so I can construct a
> whole
> >> > data flow tracing through HBase and HDFS. While, the fact is that I
> can
> >> > only get tracing data of HBase, with no data of HDFS.
> >> >
> >> > Could you give me any suggestions on how to trace the data flow in
> both
> >> > HBase and HDFS? Does anyone have similar experience? Do I need to
> modify
> >> > the source code? And maybe which part(s) should I touch? If I need to
> >> > modify the code, I will try to create a patch for that.
> >> >
> >> > Thank you.
> >> >
> >> > My Configurations:
> >> > Hadoop version: 2.6.0
> >> > HBase version: 0.99.2
> >> > HTrace version: htrace-master
> >> > OS: Ubuntu 12.04
> >> >
> >> >
> >> > Joshua
> >>
>

Re: Trace HBase/HDFS with HTrace

Posted by "Colin P. McCabe" <cm...@apache.org>.
On Wed, Feb 11, 2015 at 11:27 AM, Nick Dimiduk <nd...@gmail.com> wrote:
> I don't recall the hadoop release repo restriction being a problem, but I
> haven't tested it lately. See if you can just specify the release version
> with -Dhadoop.version or -Dhadoop-two.version.
>

Sorry, it's been a while since I did this... I guess the question is
whether 2.7.0-SNAPSHOT is available in Maven-land somewhere?  If so,
then Chunxu should forget all that stuff I said, and just build HBase
with -Dhadoop.version=2.7.0-SNAPSHOT

> I would go against branch-1.0 as this will be the eminent 1.0.0 release and
> had HTrace 3.1.0-incubating.

Thanks.

Colin


>
> -n
>
> On Wed, Feb 11, 2015 at 11:13 AM, Colin P. McCabe <cm...@apache.org>
> wrote:
>
>> Thanks for trying stuff out!  Sorry that this is a little difficult at
>> the moment.
>>
>> To really do this right, you would want to be using Hadoop with HTrace
>> 3.1.0, and HBase with HTrace 3.1.0.  Unfortunately, there hasn't been
>> a new release of Hadoop with HTrace 3.1.0.  The only existing releases
>> of Hadoop use an older version of the HTrace library.  So you will
>> have to build from source.
>>
>> If you check out Hadoop's "branch-2" branch (currently, this branch
>> represents what will be in the 2.7 release, when it is cut), and build
>> that, you will get the latest.  Then you have to build a version of
>> HBase against the version of Hadoop you have built.
>>
>> By default, HBase's Maven build will build against upstream release
>> versions of Hadoop only. So just setting
>> -Dhadoop.version=2.7.0-SNAPSHOT is not enough, since it won't know
>> where to find the jars.  To get around this problem, you can create
>> your own local maven repo. Here's how.
>>
>> In hadoop/pom.xml, add these lines to the distributionManagement stanza:
>>
>> +    <repository>
>> +      <id>localdump</id>
>> +      <url>file:///home/cmccabe/localdump/releases</url>
>> +    </repository>
>> +    <snapshotRepository>
>> +      <id>localdump</id>
>> +      <url>file:///home/cmccabe/localdump/snapshots</url>
>> +    </snapshotRepository>
>>
>> Comment out the repositories that are already there.
>>
>> Now run mkdir /home/cmccabe/localdump.
>>
>> Then, in your hadoop tree, run mvn deploy -DskipTests.
>>
>> You should get a localdump directory that has files kind of like this:
>>
>> ...
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce
>>
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/maven-metadata.xml.md5
>>
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT
>>
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml.md5
>>
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/hadoop-mapreduce-2.7.0-20121120.230341-1.pom.sha1
>>
>> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml
>> ...
>>
>> Now, add the following lines to your HBase pom.xml:
>>
>>    <repositories>
>>      <repository>
>> +      <id>localdump</id>
>> +      <url>file:///home/cmccabe/localdump</url>
>> +      <name>Local Dump</name>
>> +      <snapshots>
>> +        <enabled>true</enabled>
>> +      </snapshots>
>> +      <releases>
>> +        <enabled>true</enabled>
>> +      </releases>
>> +    </repository>
>> +    <repository>
>>
>> This will allow you to run something like:
>> mvn test -Dtest=TestMiniClusterLoadSequential -PlocalTests
>> -DredirectTestOutputToFile=true -Dhadoop.profile=2.0
>> -Dhadoop.version=2.7.0-SNAPSHOT -Dcdh.hadoop.version=2.7.0-SNAPSHOT
>>
>> Once we do a new release of Hadoop with HTrace 3.1.0 this will get a lot
>> easier.
>>
>> Related: Does anyone know what the best git branch to build from for
>> HBase would be for this kind of testing?  I've been meaning to do some
>> end to end testing (it's been on my TODO for a while)
>>
>> best,
>> Colin
>>
>> On Wed, Feb 11, 2015 at 7:55 AM, Chunxu Tang <ch...@gmail.com> wrote:
>> > Hi all,
>> >
>> > Now I’m exploiting HTrace to trace request level data flows in HBase and
>> > HDFS. I have successfully traced HBase and HDFS by using HTrace,
>> > respectively.
>> >
>> > After that, I combine HBase and HDFS together and I want to just send a
>> > PUT/GET request to HBase, but to trace the whole data flow in both HBase
>> > and HDFS. In my opinion, when I send a request such as Get to HBase, it
>> > will at last try to read the blocks on HDFS, so I can construct a whole
>> > data flow tracing through HBase and HDFS. While, the fact is that I can
>> > only get tracing data of HBase, with no data of HDFS.
>> >
>> > Could you give me any suggestions on how to trace the data flow in both
>> > HBase and HDFS? Does anyone have similar experience? Do I need to modify
>> > the source code? And maybe which part(s) should I touch? If I need to
>> > modify the code, I will try to create a patch for that.
>> >
>> > Thank you.
>> >
>> > My Configurations:
>> > Hadoop version: 2.6.0
>> > HBase version: 0.99.2
>> > HTrace version: htrace-master
>> > OS: Ubuntu 12.04
>> >
>> >
>> > Joshua
>>

Re: Trace HBase/HDFS with HTrace

Posted by Nick Dimiduk <nd...@gmail.com>.
I don't recall the hadoop release repo restriction being a problem, but I
haven't tested it lately. See if you can just specify the release version
with -Dhadoop.version or -Dhadoop-two.version.

I would go against branch-1.0 as this will be the eminent 1.0.0 release and
had HTrace 3.1.0-incubating.

-n

On Wed, Feb 11, 2015 at 11:13 AM, Colin P. McCabe <cm...@apache.org>
wrote:

> Thanks for trying stuff out!  Sorry that this is a little difficult at
> the moment.
>
> To really do this right, you would want to be using Hadoop with HTrace
> 3.1.0, and HBase with HTrace 3.1.0.  Unfortunately, there hasn't been
> a new release of Hadoop with HTrace 3.1.0.  The only existing releases
> of Hadoop use an older version of the HTrace library.  So you will
> have to build from source.
>
> If you check out Hadoop's "branch-2" branch (currently, this branch
> represents what will be in the 2.7 release, when it is cut), and build
> that, you will get the latest.  Then you have to build a version of
> HBase against the version of Hadoop you have built.
>
> By default, HBase's Maven build will build against upstream release
> versions of Hadoop only. So just setting
> -Dhadoop.version=2.7.0-SNAPSHOT is not enough, since it won't know
> where to find the jars.  To get around this problem, you can create
> your own local maven repo. Here's how.
>
> In hadoop/pom.xml, add these lines to the distributionManagement stanza:
>
> +    <repository>
> +      <id>localdump</id>
> +      <url>file:///home/cmccabe/localdump/releases</url>
> +    </repository>
> +    <snapshotRepository>
> +      <id>localdump</id>
> +      <url>file:///home/cmccabe/localdump/snapshots</url>
> +    </snapshotRepository>
>
> Comment out the repositories that are already there.
>
> Now run mkdir /home/cmccabe/localdump.
>
> Then, in your hadoop tree, run mvn deploy -DskipTests.
>
> You should get a localdump directory that has files kind of like this:
>
> ...
> /home/cmccabe/localdump/snapshots/org/apache/hadoop
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce
>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/maven-metadata.xml.md5
>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT
>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml.md5
>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/hadoop-mapreduce-2.7.0-20121120.230341-1.pom.sha1
>
> /home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml
> ...
>
> Now, add the following lines to your HBase pom.xml:
>
>    <repositories>
>      <repository>
> +      <id>localdump</id>
> +      <url>file:///home/cmccabe/localdump</url>
> +      <name>Local Dump</name>
> +      <snapshots>
> +        <enabled>true</enabled>
> +      </snapshots>
> +      <releases>
> +        <enabled>true</enabled>
> +      </releases>
> +    </repository>
> +    <repository>
>
> This will allow you to run something like:
> mvn test -Dtest=TestMiniClusterLoadSequential -PlocalTests
> -DredirectTestOutputToFile=true -Dhadoop.profile=2.0
> -Dhadoop.version=2.7.0-SNAPSHOT -Dcdh.hadoop.version=2.7.0-SNAPSHOT
>
> Once we do a new release of Hadoop with HTrace 3.1.0 this will get a lot
> easier.
>
> Related: Does anyone know what the best git branch to build from for
> HBase would be for this kind of testing?  I've been meaning to do some
> end to end testing (it's been on my TODO for a while)
>
> best,
> Colin
>
> On Wed, Feb 11, 2015 at 7:55 AM, Chunxu Tang <ch...@gmail.com> wrote:
> > Hi all,
> >
> > Now I’m exploiting HTrace to trace request level data flows in HBase and
> > HDFS. I have successfully traced HBase and HDFS by using HTrace,
> > respectively.
> >
> > After that, I combine HBase and HDFS together and I want to just send a
> > PUT/GET request to HBase, but to trace the whole data flow in both HBase
> > and HDFS. In my opinion, when I send a request such as Get to HBase, it
> > will at last try to read the blocks on HDFS, so I can construct a whole
> > data flow tracing through HBase and HDFS. While, the fact is that I can
> > only get tracing data of HBase, with no data of HDFS.
> >
> > Could you give me any suggestions on how to trace the data flow in both
> > HBase and HDFS? Does anyone have similar experience? Do I need to modify
> > the source code? And maybe which part(s) should I touch? If I need to
> > modify the code, I will try to create a patch for that.
> >
> > Thank you.
> >
> > My Configurations:
> > Hadoop version: 2.6.0
> > HBase version: 0.99.2
> > HTrace version: htrace-master
> > OS: Ubuntu 12.04
> >
> >
> > Joshua
>

Re: Trace HBase/HDFS with HTrace

Posted by "Colin P. McCabe" <cm...@apache.org>.
Thanks for trying stuff out!  Sorry that this is a little difficult at
the moment.

To really do this right, you would want to be using Hadoop with HTrace
3.1.0, and HBase with HTrace 3.1.0.  Unfortunately, there hasn't been
a new release of Hadoop with HTrace 3.1.0.  The only existing releases
of Hadoop use an older version of the HTrace library.  So you will
have to build from source.

If you check out Hadoop's "branch-2" branch (currently, this branch
represents what will be in the 2.7 release, when it is cut), and build
that, you will get the latest.  Then you have to build a version of
HBase against the version of Hadoop you have built.

By default, HBase's Maven build will build against upstream release
versions of Hadoop only. So just setting
-Dhadoop.version=2.7.0-SNAPSHOT is not enough, since it won't know
where to find the jars.  To get around this problem, you can create
your own local maven repo. Here's how.

In hadoop/pom.xml, add these lines to the distributionManagement stanza:

+    <repository>
+      <id>localdump</id>
+      <url>file:///home/cmccabe/localdump/releases</url>
+    </repository>
+    <snapshotRepository>
+      <id>localdump</id>
+      <url>file:///home/cmccabe/localdump/snapshots</url>
+    </snapshotRepository>

Comment out the repositories that are already there.

Now run mkdir /home/cmccabe/localdump.

Then, in your hadoop tree, run mvn deploy -DskipTests.

You should get a localdump directory that has files kind of like this:

...
/home/cmccabe/localdump/snapshots/org/apache/hadoop
/home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce
/home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/maven-metadata.xml.md5
/home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT
/home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml.md5
/home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/hadoop-mapreduce-2.7.0-20121120.230341-1.pom.sha1
/home/cmccabe/localdump/snapshots/org/apache/hadoop/hadoop-mapreduce/2.7.0-SNAPSHOT/maven-metadata.xml
...

Now, add the following lines to your HBase pom.xml:

   <repositories>
     <repository>
+      <id>localdump</id>
+      <url>file:///home/cmccabe/localdump</url>
+      <name>Local Dump</name>
+      <snapshots>
+        <enabled>true</enabled>
+      </snapshots>
+      <releases>
+        <enabled>true</enabled>
+      </releases>
+    </repository>
+    <repository>

This will allow you to run something like:
mvn test -Dtest=TestMiniClusterLoadSequential -PlocalTests
-DredirectTestOutputToFile=true -Dhadoop.profile=2.0
-Dhadoop.version=2.7.0-SNAPSHOT -Dcdh.hadoop.version=2.7.0-SNAPSHOT

Once we do a new release of Hadoop with HTrace 3.1.0 this will get a lot easier.

Related: Does anyone know what the best git branch to build from for
HBase would be for this kind of testing?  I've been meaning to do some
end to end testing (it's been on my TODO for a while)

best,
Colin

On Wed, Feb 11, 2015 at 7:55 AM, Chunxu Tang <ch...@gmail.com> wrote:
> Hi all,
>
> Now I’m exploiting HTrace to trace request level data flows in HBase and
> HDFS. I have successfully traced HBase and HDFS by using HTrace,
> respectively.
>
> After that, I combine HBase and HDFS together and I want to just send a
> PUT/GET request to HBase, but to trace the whole data flow in both HBase
> and HDFS. In my opinion, when I send a request such as Get to HBase, it
> will at last try to read the blocks on HDFS, so I can construct a whole
> data flow tracing through HBase and HDFS. While, the fact is that I can
> only get tracing data of HBase, with no data of HDFS.
>
> Could you give me any suggestions on how to trace the data flow in both
> HBase and HDFS? Does anyone have similar experience? Do I need to modify
> the source code? And maybe which part(s) should I touch? If I need to
> modify the code, I will try to create a patch for that.
>
> Thank you.
>
> My Configurations:
> Hadoop version: 2.6.0
> HBase version: 0.99.2
> HTrace version: htrace-master
> OS: Ubuntu 12.04
>
>
> Joshua