Re: Sources Of HBase Client Side Latency

2016-04-19 Thread Neelesh
We have very similar issues with HBase 1.1.2, especially around splits. It
takes several minutes at times. We use phoenix with two global indexes and
we know that compounds the issue a bit, but about 95% of inserts take <
50ms (with index writes included), but the remaining 5% are wild from 100ms
to tens of minutes at times

On Tue, Apr 19, 2016 at 2:07 PM, Saad Mufti  wrote:

> Hi,
>
> I found this blog post from 2014 on sources of HBase client side latency
> which I found useful:
>
>
> https://hadoop-hbase.blogspot.com/2014/08/hbase-client-response-times.html?showComment=1461099797978#c5266762058464276023
>
> Since this is a bit dated, anyone have any other sources of latency to add?
> In our experience with HBase 1.x so far we've definitely seen long
> latencies when a region server dies, but we've also seen it during
> auto-splits which this post suggests shouldn't be that long.
>
> Then we have other unexplained (so far, at least by us) big response time
> spikes and client side timeouts that can last a few minutes or in some
> cases a couple of hours that we'd like to explain. Could these be from
> either automatic major compaction or minor compactions that got upgraded to
> major?
>
> Any advice on where to start looking to investigate?
>
> Thanks.
>
> 
> Saad
>


Re: Failed open of region

2016-04-19 Thread Stack
On Tue, Apr 19, 2016 at 9:09 PM, Michal Medvecky  wrote:

> On Tue, Apr 19, 2016 at 9:02 PM, Stack  wrote:
>
> > What happens if you try to copy /hbase/data/default/MEDIA/
> > ecd1e565ab8a8bfba77cab46ed023539*
> > /F/5eacfeb8a2eb419cb6fe348df0540145 to local filesystem from HDFS (hdfs
> dfs
> > -copyLocal... or whatever it is called)?
> >
>
> Nothing, these files (and even those directories) do not exist.
>
>
NameNode is telling hbase they exist. Can you get some DEBUG in there? You
know how to set log levels? Could give us a clue.
Thanks,
St.Ack



> Your AWS giving you grief?
> >
>
> AWS is driving me crazy, but that's not for this mailinglist.
>
> Michal
>


Re: Failed open of region

2016-04-19 Thread Michal Medvecky
On Tue, Apr 19, 2016 at 9:02 PM, Stack  wrote:

> If you run hdfs fsck it shows missing blocks?
>

No, HDFS reports healthy filesystem.


> What happens if you try to copy /hbase/data/default/MEDIA/
> ecd1e565ab8a8bfba77cab46ed023539*
> /F/5eacfeb8a2eb419cb6fe348df0540145 to local filesystem from HDFS (hdfs dfs
> -copyLocal... or whatever it is called)?
>

Nothing, these files (and even those directories) do not exist.

Your AWS giving you grief?
>

AWS is driving me crazy, but that's not for this mailinglist.

Michal


Re: Failed open of region

2016-04-19 Thread Stack
If you run hdfs fsck it shows missing blocks?

What happens if you try to copy /hbase/data/default/MEDIA/
ecd1e565ab8a8bfba77cab46ed023539*
/F/5eacfeb8a2eb419cb6fe348df0540145 to local filesystem from HDFS (hdfs dfs
-copyLocal... or whatever it is called)?

Try moving aside the problematic files?

Your AWS giving you grief?
St.Ack

On Tue, Apr 19, 2016 at 8:29 PM, Michal Medvecky  wrote:

> Hello,
>
> after several network outages in AWS (never ever run HBase there!), my
> HBase was seriously damaged. After doing some steps like restarting
> namenodes, hdfs fsck, restarting all regionservers and hbase master, i'm
> still having 8 offline regions I am unable to start.
>
> When running hbck with any combination of repair parameters, it's always
> stuck on messages like:
>
> 2016-04-20 03:26:16,812 INFO  [hbasefsck-pool1-t45]
> util.HBaseFsckRepair: *Region
> still in transition, waiting for it to become assigned*: {ENCODED =>
> 8fe9d66a1f4c4739dd1929e3c38bf951, NAME =>
>
> 'MEDIA,\x01rvkUDKIuye0\x00YT,1460997677820.8fe9d66a1f4c4739dd1929e3c38bf951.',
> STARTKEY => '\x01rvkUDKIuye0\x00YT', ENDKEY =>
> '\x01stefanonoferini/club-edition-17'}
>
> when looking into regionserver logs, I see messages like:
>
> 2016-04-19 23:27:54,969 ERROR
> [RS_OPEN_REGION-prod-aws-hbase-data-0010:16020-80]
> handler.OpenRegionHandler: Failed open of region=MEDIA,\x05JEklcNpOKos\
> x00YT,1461001150488.20d48fd40c94c7c81049cbc506de4ad4., starting to roll
> back the global memstore size.
> java.io.IOException: java.io.IOException: java.io.FileNotFoundException:
> *File
> does not exist: /hbase/data/default/MEDIA/ecd1e565ab8a8bfba77cab46ed023539*
> /F/5eacfeb8a2eb419cb6fe348df0540145
> at
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:71)
> at
> org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
> at
>
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
> at
>
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
> at
>
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
> at
>
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:587)
> at
>
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB
> .java:365)
> at
>
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.j
> ava)
> at
>
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2049)
> at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2045)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at
>
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657)
> 2016-04-19 23:27:54,957 INFO
>  [StoreOpener-20d48fd40c94c7c81049cbc506de4ad4-1] hfile.CacheConfig:
> blockCache=LruBlockCache{blockCount=2, currentSize=328
> 5448, freeSize=3198122040, maxSize=3201407488, heapSize=3285448,
> minSize=3041337088, minFactor=0.95, multiSize=1520668544, multiFactor=0.5,
> singleSize=7
> 60334272, singleFactor=0.25}, cacheDataOnRead=true, cacheDataOnWrite=false,
> cacheIndexesOnWrite=false, cacheBloomsOnWrite=false,
> cacheEvictOnClose=false
> , cacheDataCompressed=false, prefetchOnOpen=false
> 2016-04-19 23:27:54,957 INFO
>  [StoreOpener-20d48fd40c94c7c81049cbc506de4ad4-1]
> compactions.CompactionConfiguration: size [134217728, 9223372036854775807
> ); files [3, 10); ratio 1.20; off-peak ratio 5.00; throttle point
> 2684354560; major period 60480, major jitter 0.50, min locality to
> com
> pact 0.70
> 2016-04-19 23:27:54,962 INFO  [StoreFileOpenerThread-F-1]
> regionserver.StoreFile$Reader: Loaded Delete Family Bloom
> (CompoundBloomFilter) metadata for 5
> eacfeb8a2eb419cb6fe348df0540145
> 2016-04-19 23:27:54,969 ERROR
> [RS_OPEN_REGION-prod-aws-hbase-data-0010:16020-80] regionserver.HRegion:
> Could not initialize all stores for the region=ME
> DIA,\x05JEklcNpOKos\x00YT,1461001150488.20d48fd40c94c7c81049cbc506de4ad4.
> 2016-04-19 23:27:54,969 WARN
>  [StoreOpener-20d48fd40c94c7c81049cbc506de4ad4-1] ipc.Client: interrupted
> waiting to send rpc request to server
> java.lang.InterruptedException
> at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:404)
> at java.util.concurrent.FutureTask.get(FutureTask.java:191)
> at
> org.apache.hadoop.ipc.Client$Connection.sendRpcRequest(Client.java:1054)
> at org.apache.hadoop.ipc.Client.call(Client.java:1449)
> 

Failed open of region

2016-04-19 Thread Michal Medvecky
Hello,

after several network outages in AWS (never ever run HBase there!), my
HBase was seriously damaged. After doing some steps like restarting
namenodes, hdfs fsck, restarting all regionservers and hbase master, i'm
still having 8 offline regions I am unable to start.

When running hbck with any combination of repair parameters, it's always
stuck on messages like:

2016-04-20 03:26:16,812 INFO  [hbasefsck-pool1-t45]
util.HBaseFsckRepair: *Region
still in transition, waiting for it to become assigned*: {ENCODED =>
8fe9d66a1f4c4739dd1929e3c38bf951, NAME =>
'MEDIA,\x01rvkUDKIuye0\x00YT,1460997677820.8fe9d66a1f4c4739dd1929e3c38bf951.',
STARTKEY => '\x01rvkUDKIuye0\x00YT', ENDKEY =>
'\x01stefanonoferini/club-edition-17'}

when looking into regionserver logs, I see messages like:

2016-04-19 23:27:54,969 ERROR
[RS_OPEN_REGION-prod-aws-hbase-data-0010:16020-80]
handler.OpenRegionHandler: Failed open of region=MEDIA,\x05JEklcNpOKos\
x00YT,1461001150488.20d48fd40c94c7c81049cbc506de4ad4., starting to roll
back the global memstore size.
java.io.IOException: java.io.IOException: java.io.FileNotFoundException: *File
does not exist: /hbase/data/default/MEDIA/ecd1e565ab8a8bfba77cab46ed023539*
/F/5eacfeb8a2eb419cb6fe348df0540145
at
org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:71)
at
org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
at
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1828)
at
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1799)
at
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1712)
at
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:587)
at
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB
.java:365)
at
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.j
ava)
at
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2049)
at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2045)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657)
2016-04-19 23:27:54,957 INFO
 [StoreOpener-20d48fd40c94c7c81049cbc506de4ad4-1] hfile.CacheConfig:
blockCache=LruBlockCache{blockCount=2, currentSize=328
5448, freeSize=3198122040, maxSize=3201407488, heapSize=3285448,
minSize=3041337088, minFactor=0.95, multiSize=1520668544, multiFactor=0.5,
singleSize=7
60334272, singleFactor=0.25}, cacheDataOnRead=true, cacheDataOnWrite=false,
cacheIndexesOnWrite=false, cacheBloomsOnWrite=false, cacheEvictOnClose=false
, cacheDataCompressed=false, prefetchOnOpen=false
2016-04-19 23:27:54,957 INFO
 [StoreOpener-20d48fd40c94c7c81049cbc506de4ad4-1]
compactions.CompactionConfiguration: size [134217728, 9223372036854775807
); files [3, 10); ratio 1.20; off-peak ratio 5.00; throttle point
2684354560; major period 60480, major jitter 0.50, min locality to
com
pact 0.70
2016-04-19 23:27:54,962 INFO  [StoreFileOpenerThread-F-1]
regionserver.StoreFile$Reader: Loaded Delete Family Bloom
(CompoundBloomFilter) metadata for 5
eacfeb8a2eb419cb6fe348df0540145
2016-04-19 23:27:54,969 ERROR
[RS_OPEN_REGION-prod-aws-hbase-data-0010:16020-80] regionserver.HRegion:
Could not initialize all stores for the region=ME
DIA,\x05JEklcNpOKos\x00YT,1461001150488.20d48fd40c94c7c81049cbc506de4ad4.
2016-04-19 23:27:54,969 WARN
 [StoreOpener-20d48fd40c94c7c81049cbc506de4ad4-1] ipc.Client: interrupted
waiting to send rpc request to server
java.lang.InterruptedException
at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:404)
at java.util.concurrent.FutureTask.get(FutureTask.java:191)
at
org.apache.hadoop.ipc.Client$Connection.sendRpcRequest(Client.java:1054)
at org.apache.hadoop.ipc.Client.call(Client.java:1449)
at org.apache.hadoop.ipc.Client.call(Client.java:1407)
at
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
at com.sun.proxy.$Proxy16.getFileInfo(Unknown Source)
at
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:771)
at sun.reflect.GeneratedMethodAccessor5.invoke(Unknown Source)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
   

Re: Sources Of HBase Client Side Latency

2016-04-19 Thread Stack
On Tue, Apr 19, 2016 at 2:07 PM, Saad Mufti  wrote:

> Hi,
>
> I found this blog post from 2014 on sources of HBase client side latency
> which I found useful:
>
>
> https://hadoop-hbase.blogspot.com/2014/08/hbase-client-response-times.html?showComment=1461099797978#c5266762058464276023
>
>
It is written by an authority.



> Since this is a bit dated, anyone have any other sources of latency to add?
> In our experience with HBase 1.x so far we've definitely seen long
> latencies when a region server dies, but we've also seen it during
> auto-splits which this post suggests shouldn't be that long.
>
>
Easy enough to check. Look at the master log and see the steps involved.



> Then we have other unexplained (so far, at least by us) big response time
> spikes and client side timeouts that can last a few minutes or in some
> cases a couple of hours that we'd like to explain. Could these be from
> either automatic major compaction or minor compactions that got upgraded to
> major?
>
>
These may up latency when running but not for minutes or hours.

Sounds like something else is going on.



> Any advice on where to start looking to investigate?
>
>
Check master log at the time of slowness and then follow your nose. If you
need more specifics on how to debug, come back here w/ some more detail
around a particular event.

Yours,
St.Ack


> Thanks.
>
> 
> Saad
>


Re: Hbase Replication no longer replicating, help diagnose

2016-04-19 Thread Abraham Tom
my timeout is set pretty high

120

maybe too high
We do get bursts of large changes when I update hbase via hive map reduce

I restarted both clusters and they caught up - but after a couple of days
they just slow down and stop

On Fri, Apr 15, 2016 at 5:49 AM, ashish singhi 
wrote:

> Let me explain in theory how it works (considering default configuration
> values)
>
> Assume 1 peer RS is already handling
> 3(hbase.regionserver.replication.handler.count) replication requests and it
> was not completed within 1 minute(hbase.rpc.timeout) time (due to some
> unknown reasons, may be slow rs or network speed...) then source RS will
> get CallTimeOutException and it will resend this request again to the same
> peer RS, so now this requests will be added in this peer RS queue (Max
> queue size = 30,
> hbase.regionserver.replication.handler.count*hbase.ipc.server.max.callqueue.length).
> Both running and waiting requests size will be counted for callQueueSize,
> so (running + waiting requests)*64MB(replication.source.size.capacity) will
> cross the call queue size 1GB(hbase.ipc.server.max.callqueue.size) and will
> result into CallQueueTooBigException exception.
>
> Now why those running requests are not getting completed, I assume this
> can be a reason, 1 peer RS received a replication request and it internally
> distributes this batch call to other RS in the peer cluster and this may
> get stuck as other peer RS also would have received replication request
> from other source cluster RS... so it might result in a kind off dead lock,
> where 1 peer RS is waiting for another peer RS to finish the request and
> that RS in turn might be processing some other request and waiting for its
> completion.
>
> So to avoid this problem, we need to find out the cause why peer RS is
> slow ? Based on that and network speed, need to adjust the
> hbase.rpc.timeout value and restart the source and peer cluster.
>
> Regards,
> Ashish
>
> -Original Message-
> From: Abraham Tom [mailto:work2m...@gmail.com]
> Sent: 14 April 2016 18:52
> To: Hbase-User
> Subject: Hbase Replication no longer replicating, help diagnose
>
> my hbase replication has stopped
>
> I am on hbase version 1.0.0-cdh5.4.8 (Cloudera build)
>
> I have 2 clusters in 2 different datacenters
>
> 1 is master the other is slave
>
>
>
> I see the following errors in log
>
>
>
> 2016-04-13 22:32:50,217 WARN
>
> org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint:
> Can't replicate because of a local or network error:
> java.io.IOException: Call to
> hadoop2-private.sjc03.infra.com/10.160.22.99:60020 failed on local
> exception: org.apache.hadoop.hbase.ipc.CallTimeoutException: Call id=1014,
> waitTime=121, operationTimeout=120 expired.
> at
> org.apache.hadoop.hbase.ipc.RpcClientImpl.wrapException(RpcClientImpl.java:1255)
> at
> org.apache.hadoop.hbase.ipc.RpcClientImpl.call(RpcClientImpl.java:1223)
> at
> org.apache.hadoop.hbase.ipc.AbstractRpcClient.callBlockingMethod(AbstractRpcClient.java:216)
> at
> org.apache.hadoop.hbase.ipc.AbstractRpcClient$BlockingRpcChannelImplementation.callBlockingMethod(AbstractRpcClient.java:300)
> at
> org.apache.hadoop.hbase.protobuf.generated.AdminProtos$AdminService$BlockingStub.replicateWALEntry(AdminProtos.java:21783)
> at
> org.apache.hadoop.hbase.protobuf.ReplicationProtbufUtil.replicateWALEntry(ReplicationProtbufUtil.java:65)
> at
> org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint.replicate(HBaseInterClusterReplicationEndpoint.java:161)
> at
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSource.shipEdits(ReplicationSource.java:696)
> at
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSource.run(ReplicationSource.java:410)
> Caused by: org.apache.hadoop.hbase.ipc.CallTimeoutException: Call id=1014,
> waitTime=121, operationTimeout=120 expired.
> at
> org.apache.hadoop.hbase.ipc.Call.checkAndSetTimeout(Call.java:70)
> at
> org.apache.hadoop.hbase.ipc.RpcClientImpl.call(RpcClientImpl.java:1197)
> ... 7 more
>
>
>
>
>
> which in turn fills the queue and I get
>
> 2016-04-13 22:35:19,555 WARN
>
> org.apache.hadoop.hbase.replication.regionserver.HBaseInterClusterReplicationEndpoint:
> Can't replicate because of an error on the remote cluster:
>
> org.apache.hadoop.hbase.ipc.RemoteWithExtrasException(org.apache.hadoop.hbase.ipc.RpcServer$CallQueueTooBigException):
> Call queue is full on /0.0.0.0:60020, is
> hbase.ipc.server.max.callqueue.size too small?
> at
> org.apache.hadoop.hbase.ipc.RpcClientImpl.call(RpcClientImpl.java:1219)
> at
> org.apache.hadoop.hbase.ipc.AbstractRpcClient.callBlockingMethod(AbstractRpcClient.java:216)
> at
> org.apache.hadoop.hbase.ipc.AbstractRpcClient$BlockingRpcChannelImplementation.callBlockingMethod(AbstractRpcClient.java:300)
> at
> org.apache.hado

Sources Of HBase Client Side Latency

2016-04-19 Thread Saad Mufti
Hi,

I found this blog post from 2014 on sources of HBase client side latency
which I found useful:

https://hadoop-hbase.blogspot.com/2014/08/hbase-client-response-times.html?showComment=1461099797978#c5266762058464276023

Since this is a bit dated, anyone have any other sources of latency to add?
In our experience with HBase 1.x so far we've definitely seen long
latencies when a region server dies, but we've also seen it during
auto-splits which this post suggests shouldn't be that long.

Then we have other unexplained (so far, at least by us) big response time
spikes and client side timeouts that can last a few minutes or in some
cases a couple of hours that we'd like to explain. Could these be from
either automatic major compaction or minor compactions that got upgraded to
major?

Any advice on where to start looking to investigate?

Thanks.


Saad


Re: Major Compaction Strategy

2016-04-19 Thread Ted Yu
Please use the following method of HBaseAdmin:

  public CompactionState getCompactionStateForRegion(final byte[]
regionName)

Cheers

On Tue, Apr 19, 2016 at 12:56 PM, Saad Mufti  wrote:

> Hi,
>
> We have a large HBase 1.x cluster in AWS and have disabled automatic major
> compaction as advised. We were running our own code for compaction daily
> around midnight which calls HBaseAdmin.majorCompactRegion(byte[]
> regionName) in a rolling fashion across all regions.
>
> But we missed the fact that this is an asynchronous operation, so in
> practice this causes major compaction to run across all regions, at least
> those not already major compacted (for example because previous minor
> compactions got upgraded to major ones).
>
> We don't really have a suitable low load period, so what is a suitable way
> to make major compaction run in a rolling fashion region by region? The API
> above provides no return value for us to be able to wait for one compaction
> to finish before moving to the next.
>
> Thanks.
>
> 
> Saad
>


Major Compaction Strategy

2016-04-19 Thread Saad Mufti
Hi,

We have a large HBase 1.x cluster in AWS and have disabled automatic major
compaction as advised. We were running our own code for compaction daily
around midnight which calls HBaseAdmin.majorCompactRegion(byte[]
regionName) in a rolling fashion across all regions.

But we missed the fact that this is an asynchronous operation, so in
practice this causes major compaction to run across all regions, at least
those not already major compacted (for example because previous minor
compactions got upgraded to major ones).

We don't really have a suitable low load period, so what is a suitable way
to make major compaction run in a rolling fashion region by region? The API
above provides no return value for us to be able to wait for one compaction
to finish before moving to the next.

Thanks.


Saad


Re: Processing rows in parallel with MapReduce jobs.

2016-04-19 Thread Ted Yu
>From the error, you need to provide an argumentless ctor for
MyTableInputFormat.

On Tue, Apr 19, 2016 at 12:12 AM, Ivan Cores gonzalez 
wrote:

>
> Hi Ted,
>
> Sorry, I forgot to write the error. In runtime I have the next exception:
>
> Exception in thread "main" java.lang.RuntimeException:
> java.lang.NoSuchMethodException:
> simplerowcounter.SimpleRowCounter$MyTableInputFormat.()
>
> the program works fine if I don't use "MyTableInputFormat" modifying the
> call to initTableMapperJob:
>
> TableMapReduceUtil.initTableMapperJob(tableName, scan,
> RowCounterMapper.class,
> ImmutableBytesWritable.class, Result.class, job);   // -->
> works fine without MyTableInputFormat
>
> That's why I asked If you see any problem in the code. Because maybe I
> forgot override some method or something is missing.
>
> Best,
> Iván.
>
>
> - Mensaje original -
> > De: "Ted Yu" 
> > Para: user@hbase.apache.org
> > Enviados: Martes, 19 de Abril 2016 0:22:05
> > Asunto: Re: Processing rows in parallel with MapReduce jobs.
> >
> > Did you see the "Message to log?" log ?
> >
> > Can you pastebin the error / exception you got ?
> >
> > On Mon, Apr 18, 2016 at 1:54 AM, Ivan Cores gonzalez <
> ivan.co...@inria.fr>
> > wrote:
> >
> > >
> > >
> > > Hi Ted,
> > > So, If I understand the behaviour of getSplits(), I can create
> "virtual"
> > > splits overriding the getSplits function.
> > > I was performing some tests, but my code crash in runtime and I cannot
> > > found the problem.
> > > Any help? I didn't find examples.
> > >
> > >
> > > public class SimpleRowCounter extends Configured implements Tool {
> > >
> > >   static class RowCounterMapper extends
> > > TableMapper {
> > > public static enum Counters { ROWS }
> > > @Override
> > > public void map(ImmutableBytesWritable row, Result value, Context
> > > context) {
> > >   context.getCounter(Counters.ROWS).increment(1);
> > > try {
> > > Thread.sleep(3000); //Simulates work
> > > } catch (InterruptedException name) { }
> > > }
> > >   }
> > >
> > >   public class MyTableInputFormat extends TableInputFormat {
> > > @Override
> > > public List getSplits(JobContext context) throws
> > > IOException {
> > > //Just to detect if this method is being called ...
> > > List splits = super.getSplits(context);
> > > System.out.printf("Message to log? \n" );
> > > return splits;
> > > }
> > >   }
> > >
> > >   @Override
> > >   public int run(String[] args) throws Exception {
> > > if (args.length != 1) {
> > >   System.err.println("Usage: SimpleRowCounter ");
> > >   return -1;
> > > }
> > > String tableName = args[0];
> > >
> > > Scan scan = new Scan();
> > > scan.setFilter(new FirstKeyOnlyFilter());
> > > scan.setCaching(500);
> > > scan.setCacheBlocks(false);
> > >
> > > Job job = new Job(getConf(), getClass().getSimpleName());
> > > job.setJarByClass(getClass());
> > >
> > > TableMapReduceUtil.initTableMapperJob(tableName, scan,
> > > RowCounterMapper.class,
> > > ImmutableBytesWritable.class, Result.class, job, true,
> > > MyTableInputFormat.class);
> > >
> > > job.setNumReduceTasks(0);
> > > job.setOutputFormatClass(NullOutputFormat.class);
> > > return job.waitForCompletion(true) ? 0 : 1;
> > >   }
> > >
> > >   public static void main(String[] args) throws Exception {
> > > int exitCode = ToolRunner.run(HBaseConfiguration.create(),
> > > new SimpleRowCounter(), args);
> > > System.exit(exitCode);
> > >   }
> > > }
> > >
> > > Thanks so much,
> > > Iván.
> > >
> > >
> > >
> > >
> > > - Mensaje original -
> > > > De: "Ted Yu" 
> > > > Para: user@hbase.apache.org
> > > > Enviados: Martes, 12 de Abril 2016 17:29:52
> > > > Asunto: Re: Processing rows in parallel with MapReduce jobs.
> > > >
> > > > Please take a look at TableInputFormatBase#getSplits() :
> > > >
> > > >* Calculates the splits that will serve as input for the map
> tasks.
> > > The
> > > >
> > > >* number of splits matches the number of regions in a table.
> > > >
> > > > Each mapper would be reading one of the regions.
> > > >
> > > > On Tue, Apr 12, 2016 at 8:18 AM, Ivan Cores gonzalez <
> > > ivan.co...@inria.fr>
> > > > wrote:
> > > >
> > > > > Hi Ted,
> > > > > Yes, I mean same region.
> > > > >
> > > > > I wasn't using the getSplits() function. I'm trying to add it to my
> > > code
> > > > > but I'm not sure how I have to do it. Is there any example in the
> > > website?
> > > > > I can not find anything. (By the way, I'm using TableInputFormat,
> not
> > > > > InputFormat)
> > > > >
> > > > > But just to confirm, with the getSplits() function, Are mappers
> > > processing
> > > > > rows in the same region executed in parallel? (assuming that there
> are
> > > > > empty
> > > > > processors/cores)
> > > > >
> > > > > Thanks,
> > > > > Ivan.
> > > > >
> 

Re: Processing rows in parallel with MapReduce jobs.

2016-04-19 Thread Ivan Cores gonzalez

Hi Ted,

Sorry, I forgot to write the error. In runtime I have the next exception:

Exception in thread "main" java.lang.RuntimeException: 
java.lang.NoSuchMethodException: 
simplerowcounter.SimpleRowCounter$MyTableInputFormat.()

the program works fine if I don't use "MyTableInputFormat" modifying the call 
to initTableMapperJob:

TableMapReduceUtil.initTableMapperJob(tableName, scan, 
RowCounterMapper.class, 
ImmutableBytesWritable.class, Result.class, job);   // --> 
works fine without MyTableInputFormat

That's why I asked If you see any problem in the code. Because maybe I forgot 
override some method or something is missing.

Best,
Iván.


- Mensaje original -
> De: "Ted Yu" 
> Para: user@hbase.apache.org
> Enviados: Martes, 19 de Abril 2016 0:22:05
> Asunto: Re: Processing rows in parallel with MapReduce jobs.
> 
> Did you see the "Message to log?" log ?
> 
> Can you pastebin the error / exception you got ?
> 
> On Mon, Apr 18, 2016 at 1:54 AM, Ivan Cores gonzalez 
> wrote:
> 
> >
> >
> > Hi Ted,
> > So, If I understand the behaviour of getSplits(), I can create "virtual"
> > splits overriding the getSplits function.
> > I was performing some tests, but my code crash in runtime and I cannot
> > found the problem.
> > Any help? I didn't find examples.
> >
> >
> > public class SimpleRowCounter extends Configured implements Tool {
> >
> >   static class RowCounterMapper extends
> > TableMapper {
> > public static enum Counters { ROWS }
> > @Override
> > public void map(ImmutableBytesWritable row, Result value, Context
> > context) {
> >   context.getCounter(Counters.ROWS).increment(1);
> > try {
> > Thread.sleep(3000); //Simulates work
> > } catch (InterruptedException name) { }
> > }
> >   }
> >
> >   public class MyTableInputFormat extends TableInputFormat {
> > @Override
> > public List getSplits(JobContext context) throws
> > IOException {
> > //Just to detect if this method is being called ...
> > List splits = super.getSplits(context);
> > System.out.printf("Message to log? \n" );
> > return splits;
> > }
> >   }
> >
> >   @Override
> >   public int run(String[] args) throws Exception {
> > if (args.length != 1) {
> >   System.err.println("Usage: SimpleRowCounter ");
> >   return -1;
> > }
> > String tableName = args[0];
> >
> > Scan scan = new Scan();
> > scan.setFilter(new FirstKeyOnlyFilter());
> > scan.setCaching(500);
> > scan.setCacheBlocks(false);
> >
> > Job job = new Job(getConf(), getClass().getSimpleName());
> > job.setJarByClass(getClass());
> >
> > TableMapReduceUtil.initTableMapperJob(tableName, scan,
> > RowCounterMapper.class,
> > ImmutableBytesWritable.class, Result.class, job, true,
> > MyTableInputFormat.class);
> >
> > job.setNumReduceTasks(0);
> > job.setOutputFormatClass(NullOutputFormat.class);
> > return job.waitForCompletion(true) ? 0 : 1;
> >   }
> >
> >   public static void main(String[] args) throws Exception {
> > int exitCode = ToolRunner.run(HBaseConfiguration.create(),
> > new SimpleRowCounter(), args);
> > System.exit(exitCode);
> >   }
> > }
> >
> > Thanks so much,
> > Iván.
> >
> >
> >
> >
> > - Mensaje original -
> > > De: "Ted Yu" 
> > > Para: user@hbase.apache.org
> > > Enviados: Martes, 12 de Abril 2016 17:29:52
> > > Asunto: Re: Processing rows in parallel with MapReduce jobs.
> > >
> > > Please take a look at TableInputFormatBase#getSplits() :
> > >
> > >* Calculates the splits that will serve as input for the map tasks.
> > The
> > >
> > >* number of splits matches the number of regions in a table.
> > >
> > > Each mapper would be reading one of the regions.
> > >
> > > On Tue, Apr 12, 2016 at 8:18 AM, Ivan Cores gonzalez <
> > ivan.co...@inria.fr>
> > > wrote:
> > >
> > > > Hi Ted,
> > > > Yes, I mean same region.
> > > >
> > > > I wasn't using the getSplits() function. I'm trying to add it to my
> > code
> > > > but I'm not sure how I have to do it. Is there any example in the
> > website?
> > > > I can not find anything. (By the way, I'm using TableInputFormat, not
> > > > InputFormat)
> > > >
> > > > But just to confirm, with the getSplits() function, Are mappers
> > processing
> > > > rows in the same region executed in parallel? (assuming that there are
> > > > empty
> > > > processors/cores)
> > > >
> > > > Thanks,
> > > > Ivan.
> > > >
> > > >
> > > > - Mensaje original -
> > > > > De: "Ted Yu" 
> > > > > Para: user@hbase.apache.org
> > > > > Enviados: Lunes, 11 de Abril 2016 15:10:29
> > > > > Asunto: Re: Processing rows in parallel with MapReduce jobs.
> > > > >
> > > > > bq. if they are located in the same split?
> > > > >
> > > > > Probably you meant same region.
> > > > >
> > > > > Can you show the getSplits() for the InputFormat of your MapReduce
> > job ?
> > > > >
> > > >