Khaled Elmeleegy | 2 Oct 09:12 2014
Picon

HBase read performance

Hi,

I am trying to do a scatter/gather on hbase (0.98.6.1), where I have a client reading ~1000 keys from an HBase
table. These keys happen to fall on the same region server. For my reads I use reverse scan to read each key as
I want the key prior to a specific time stamp (time stamps are stored in reverse order). I don't believe gets
can accomplish that, right? so I use scan, with caching set to 1.

I use 2000 reader threads in the client and on HBase, I've set hbase.regionserver.handler.count to
1500. With this setup, my scatter gather is very slow and can take up to 10s in total. Timing an individual
getScanner(..) call on the client side, it can easily take few hundreds of ms. I also got the following
metrics from the region server in question:

"queueCallTime_mean" : 2.190855525775637,
"queueCallTime_median" : 0.0,
"queueCallTime_75th_percentile" : 0.0,
"queueCallTime_95th_percentile" : 1.0,
"queueCallTime_99th_percentile" : 556.9799999999818,

"processCallTime_min" : 0,
"processCallTime_max" : 12755,
"processCallTime_mean" : 105.64873440912682,
"processCallTime_median" : 0.0,
"processCallTime_75th_percentile" : 2.0,
"processCallTime_95th_percentile" : 7917.95,
"processCallTime_99th_percentile" :
8876.89,

"namespace_default_table_delta_region_87be70d7710f95c05cfcc90181d183b4_metric_scanNext_min"
:
89,
(Continue reading)

Tao Xiao | 2 Oct 05:27 2014
Picon

How to make a given table spread evenly across the cluster

Hi all,

I have a HBase table containing 40 million records. Checking HBase UI, I
could see that this table was not spread evenly across the cluster. Two
nodes, *ec1.hadoop.com:60020 <http://ec1.hadoop.com:60020>* and
*ec5.hadoop.com:60020
<http://ec5.hadoop.com:60020>*, for example, has only one region each,
while other nodes have more regions. Here is the screen shot
<http://imgbin.org/index.php?page=image&id=19571>.

I tried to split that table by clicking the button "Split". Later, each
region of that table was split into two regions, but all regions for that
table as a whole were still unevenly spread across the cluster. Here
is a screen
shot after I click the button "Split"
<http://imgbin.org/index.php?page=image&id=19572>.

So I'd like to know how to adjust the regions for a table so that all
regions for that table can be evenly spread across the cluster.

Thanks
Invkrh | 1 Oct 16:25 2014
Picon

can not start backup master on slave node

Hi,

I am running a hbase cluster on Amazon EC2.

*HBase Version = 0.98.4-hadoop1*

The cluster is on fully distributed mode with one master and three slaves.

On the master:

slaves' host names are added to conf/backup-masters
$HBASE_HOME/conf is rsync'ed on cluster

The hbase is started normally, but the backup-master process is not started
on slaves node.

"$ jps" on slaves nodes does not show a HMaster process.

After checking log, I found the following:

2014-10-01 14:00:22,864 FATAL [master:ip-10-239-180-222:*60000*]
master.HMaster: Unhandled exception. Starting shutdown.
java.net.BindException: *Address already in use*
	at sun.nio.ch.Net.bind0(Native Method)
	at sun.nio.ch.Net.bind(Net.java:444)
	at sun.nio.ch.Net.bind(Net.java:436)
	at
sun.nio.ch.ServerSocketChannelImpl.bind(ServerSocketChannelImpl.java:214)
	at sun.nio.ch.ServerSocketAdaptor.bind(ServerSocketAdaptor.java:74)
	at
(Continue reading)

Ian Brooks | 1 Oct 13:37 2014

protobuf error

Hi,

 I have a java client that connects to hbase and reads and writes data to hbase. every now and then, I'm seeing
the following stack traces in the application log and I'm not sure why they are coming up.

org.apache.hadoop.hbase.client.ClusterStatusListener - ERROR - Unexpected exception, continuing.
com.google.protobuf.InvalidProtocolBufferException: Protocol message tag had invalid wire type.
        at com.google.protobuf.InvalidProtocolBufferException.invalidWireType(InvalidProtocolBufferException.java:99)
        at com.google.protobuf.UnknownFieldSet$Builder.mergeFieldFrom(UnknownFieldSet.java:498)
        at com.google.protobuf.GeneratedMessage.parseUnknownField(GeneratedMessage.java:193)
        at org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos$ClusterStatus.<init>(ClusterStatusProtos.java:7554)
        at org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos$ClusterStatus.<init>(ClusterStatusProtos.java:7512)
        at org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos$ClusterStatus$1.parsePartialFrom(ClusterStatusProtos.java:7689)
        at org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos$ClusterStatus$1.parsePartialFrom(ClusterStatusProtos.java:7684)
        at com.google.protobuf.AbstractParser.parsePartialFrom(AbstractParser.java:141)
        at com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:176)
        at com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:182)
        at com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:49)
        at org.jboss.netty.handler.codec.protobuf.ProtobufDecoder.decode(ProtobufDecoder.java:122)
        at org.jboss.netty.handler.codec.oneone.OneToOneDecoder.handleUpstream(OneToOneDecoder.java:66)
        at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:268)
        at org.jboss.netty.channel.socket.oio.OioDatagramWorker.process(OioDatagramWorker.java:52)
        at org.jboss.netty.channel.socket.oio.AbstractOioWorker.run(AbstractOioWorker.java:73)
        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)

I'm running hbase-0.98.3-hadoop2 

-Ian
(Continue reading)

Ted | 30 Sep 23:19 2014
Picon

are column qualifiers safe as user inputed values?

Hi I'm wondering if it's safe to use user inputed values as column qualifiers.

I realised there maybe a sensible size limit, but that's easily checked.

The scenario is if you wanted to store simple key/value pairs into
column/values like perhaps some ones preferences like :

FavouriteColour=Red
FavouritePet=Cat

where the user may get to choose both the key and value.

Basically the concern is special characters and or special parsing of
the column names, as an example the column names are allegedly =
<family_name> : <column_qualifier>

so what happens if people put more colons in the qualifier and or
escape characters like backspace or other control characters etc? Is
there any danger or is it all just uninterpreted bytes values after
the first colon?

thanks
--

-- 
Ted.

innowireless TaeYun Kim | 30 Sep 14:21 2014
Picon

HBase 0.98.6.1 maven dependency

Hi,

I'm relatively new to HBase and maven.

Currently I'm trying to add dependency for hbase-0.98.6.1-hadoop2.jar to my
application.

But when I run 'mvn package' after adding the dependency,  it fails with the
following error:

[ERROR] Failed to execute goal on project MyApp: Could not resolve
dependencies for project com.innowireless.xcapvuze:MyApp:jar:0.0.1-SNAPSHOT:
Failure to find org.apache.hbase:hbase:jar:0.98.6.1-hadoop2 in
http://repo.maven.apache.org/maven2 was cached in the local repository,
resolution will not be reattempted until the update interval of central has
elapsed or updates are forced -> [Help 1]

'mvn package -U' also failed.

When I inspected the local repository in
.m2\repository\org\apache\hbase\hbase\0.98.6.1-hadoop2, there was no jar
file, while 0.94.6 directory has one.

How can I get the 0.98.6.1 jar?

Should I build it myself?

Or should I add each 'component' jars (hbase-client, hbase-common,
hbase-protocol.) to the dependency?

(Continue reading)

Vikram Singh Chandel | 30 Sep 13:52 2014
Picon

Getting Class not Found Exception while attaching CoProcessor jar( in HDFS) to table

Hi

*HBase : 0.98.1 CDH 5.1.1*

When i am trying to attach CoPro jar to table in RS logs i am getting
following Exceptions

ERROR org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost:* Failed
to load coprocessor *
org.apache.hadoop.hbase.coprocessor.RowCountEndpointCoPro
java.io.IOException: *Cannot load external coprocessor class *
org.apache.hadoop.hbase.coprocessor.RowCountEndpointCoPro
        at
org.apache.hadoop.hbase.coprocessor.CoprocessorHost.load(CoprocessorHost.java:208)
        at
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.loadTableCoprocessors(RegionCoprocessorHost.java:207)
        at
org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.<init>(RegionCoprocessorHost.java:163)
        at
org.apache.hadoop.hbase.regionserver.HRegion.<init>(HRegion.java:623)
        at
org.apache.hadoop.hbase.regionserver.HRegion.<init>(HRegion.java:530)
        at sun.reflect.GeneratedConstructorAccessor15.newInstance(Unknown
Source)
        at
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
        at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
        at
org.apache.hadoop.hbase.regionserver.HRegion.newHRegion(HRegion.java:4137)
        at
(Continue reading)

Brian Jeltema | 30 Sep 13:33 2014
Picon

ExportSnapshot webhdfs problems

I’m trying to use ExportSnapshot to copy a snapshot from a Hadoop 1 to a Hadoop 2 cluster using the webhdfs protocol.
I’ve done this successfully before, though there are always mapper failures and retries in the job log.
However, I’m not
having success with a rather large table due to an excessive number of failures. The exceptions in the job
log are always:

14/09/29 20:28:11 INFO mapred.JobClient: Task Id : attempt_201409241055_0024_m_000005_1, Status : FAILED
org.apache.hadoop.ipc.RemoteException
	at org.apache.hadoop.hdfs.web.JsonUtil.toRemoteException(JsonUtil.java:114)
	at org.apache.hadoop.hdfs.web.WebHdfsFileSystem.validateResponse(WebHdfsFileSystem.java:290)
	at org.apache.hadoop.hdfs.web.WebHdfsFileSystem.access$500(WebHdfsFileSystem.java:98)
	at org.apache.hadoop.hdfs.web.WebHdfsFileSystem$2.close(WebHdfsFileSystem.java:653)
	at org.apache.hadoop.hbase.snapshot.ExportSnapshot$ExportMapper.copyFile(ExportSnapshot.java:204)
	at org.apache.hadoop.hbase.snapshot.ExportSnapshot$ExportMapper.map(ExportSnapshot.java:146)
        …
        …
        …

So I presume the real exception is taking place on the target system. However, examining the namenode logs
and a handful of
the datanode logs has not revealed any exceptions that correlate with those in the job log. Is there some
other log I should be looking at?
I reduced the number of mappers to 6 and the target cluster has 10 datanodes, so it’s hard to believe its a
capacity problem.

Thanks
Brian
SACHINGUPTA | 29 Sep 14:43 2014

Access hbase remotely from java client

Hello guys

I am using the hbase java api to connect to hbase remotely, but when I 
executed the java code, got |MasterNotRunningException|. When I debugged 
the code, I came to know that zookeeper was returning the address of 
hmaster as localhost.localdomain, so the client was trying to search for 
the hmaster locally. When I changed the |/etc/hosts| file of my local 
machine from where i am running the java client as:

|<ip of the master>  localhost.localdomain|

then it worked fine.

However, I think that this is not the right way. I think I have to 
change the addresses somewhere in the configuration of zookeeper, but I 
did not get it.

please help
thanks

--

-- 
Thanks
Sachin Gupta

Vikram Singh Chandel | 29 Sep 10:48 2014
Picon

Need help in HBase 0.98.1 CoProcessor Execution

Hi

We are trying to migrating to* HBase 0.98.1(CDH 5.1.1) from 0.94.6*,
to use *Bucket
Cache + CoProcessor* and to check the performance improvement but looking
into the API i found that a lot has changed.

I tried using the HBase-example jar for the row count coprocessor, the
coprocessor jar contains the *rowcount endpoint and the Example protos(do i
need to add anything else) *and i used TestRowCountEndpoint code as my
client (added a main method to call Coprocessor Service)

Table is spilt on 13 regions over a 4 node cluster (POC test cluster)

Getting following exceptions:

*RS1 (Region Server 1)*
Unexpected throwable object
com.google.protobuf.UninitializedMessageException: Message missing required
fields: count
at
com.google.protobuf.AbstractMessage$Builder.newUninitializedMessageException(AbstractMessage.java:770)
at
org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos$CountResponse$Builder.build(ExampleProtos.java:684)
at
org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos$CountResponse$Builder.build(ExampleProtos.java:628)
at
org.apache.hadoop.hbase.regionserver.HRegion.execService(HRegion.java:5554)
at
org.apache.hadoop.hbase.regionserver.HRegionServer.execServiceOnRegion(HRegionServer.java:3300)
(Continue reading)

Henry Hung | 29 Sep 09:05 2014

scan + filter failed with OutOfOrderScannerNextException

Hi All,

Is there a way to let scanner finish scanning all regions without throwing this kind of error? I'm using scan
with filter MUST_PASS_ALL, and I observe that whenever the result data is smaller (let's say 10%) compare
to another filter with larger result (let's say 80%), it always failed.

org.apache.hadoop.hbase.DoNotRetryIOException: Failed after retry of
OutOfOrderScannerNextException: was there a rpc timeout?
at org.apache.hadoop.hbase.client.ClientScanner.next(ClientScanner.java:391)
at com.winbond.hbase.schema.mesperflogtime.MesPerfLogTimeResultSet.next(MesPerfLogTimeResultSet.java:58)
at com.winbond.hbase.schema.mesperflogtime.MesPerfLogTimeResultSet.next(MesPerfLogTimeResultSet.java:1)
at com.winbond.hbase.executor.Program.main(Program.java:106)
Caused by: org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException:
org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException: Expected nextCallSeq: 1
But the nextCallSeq got from client: 0; request=scanner_id: 1594996183743944787 number_of_rows: 100
close_scanner: false next_call_seq: 0
at org.apache.hadoop.hbase.regionserver.HRegionServer.scan(HRegionServer.java:3007)
at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:26929)
at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2146)
at org.apache.hadoop.hbase.ipc.RpcServer$Handler.run(RpcServer.java:1851)

at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:39)
at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)
at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
at org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
at org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:95)
at org.apache.hadoop.hbase.protobuf.ProtobufUtil.getRemoteException(ProtobufUtil.java:235)
at org.apache.hadoop.hbase.client.ScannerCallable.call(ScannerCallable.java:197)
at org.apache.hadoop.hbase.client.ScannerCallable.call(ScannerCallable.java:57)
(Continue reading)


Gmane