I encountered a similar problem in my  CP. I think the issue is, you seem 
to use YCSB to put a lot of records in a short period of time. By default 
YCSB has a big write buffer. In the CP you do not set autoflush so by 
default it is true. Therefore a lot of incoming puts to the CP are waiting 
for the chained put (seems to be an index put) to be completed -- this is 
causing the timeout.


Best Regards,
Wei

Wei Tan 
Research Staff Member 
IBM T. J. Watson Research Center
19 Skyline Dr, Hawthorne, NY  10532
w...@us.ibm.com; 914-784-6752



From:   Henry JunYoung KIM <henry.jy...@gmail.com>
To:     user@hbase.apache.org, 
Date:   08/13/2012 01:15 AM
Subject:        Re: Coprocessor tests under busy insertions



hi, Anoop.

this is my implementation using Coprocessors RegionObserver.

…
@Override
    public void prePut(ObserverContext<RegionCoprocessorEnvironment> e, 
Put put, WALEdit edit,
            boolean writeToWAL) throws IOException {
        String tableName = 
e.getEnvironment().getRegion().getRegionInfo().getTableNameAsString()
                .toLowerCase();
 
        if (tableName.equals(BLOG)) {
            //HTableInterface table = pool.getTable(SIDX);
            HTableInterface table = 
e.getEnvironment().getTable(Bytes.toBytes(SIDX));

            if (table == null) {
                log.error("failed to get a connection.");
                return;
            }

            try {
                Map<byte[], List<KeyValue>> familyMap = 
put.getFamilyMap();

                List<KeyValue> kvs = familyMap.get(COLUMN_FAMILY_BYTES);

                if (kvs != null) {
                    for (KeyValue kv : kvs) {
                        if 
(StringUtils.equals(Bytes.toString(kv.getQualifier()), "field0")) {
                            byte[] row = put.getRow();
                            Put idx = new Put(row);
                            idx.add(COLUMN_FAMILY_BYTES, 
"field0".getBytes(), kv.getValue());
                            table.put(idx);
                        }
                    }
                }
            } catch (Exception ex) {
                log.error("coprocessor error : ", ex);
            } finally {
                table.close();
            }
        }
    }
…

thanks for your response.

- Henry

2012. 8. 13., 오후 1:01, Anoop Sam John <anoo...@huawei.com> 작성:

> Can u paste your CP implementation here [prePut/ postPut?]
> Are u doing check for the table in CP hook? U need to only handle the 
hooks while it is being called for your table. Remember that your index 
table also have these same hooks.
> 
> -Anoop-
> ________________________________________
> From: Henry JunYoung KIM [henry.jy...@gmail.com]
> Sent: Monday, August 13, 2012 7:18 AM
> To: user@hbase.apache.org
> Subject: Coprocessor tests under busy insertions
> 
> Hi, hbase users.
> 
> now, I am testing coprocessors to create secondary indexes in 
background.
> coprocessors itself is packaged in base 0.92.1 I am using.
> 
> the scenario I want to describe is this one.
> 
> the main table is 'blog' which is having a field named 'userId'.
> from this field I want to create secondary index to map 'userId' and 
it's 'url'.
> 
> I put RegionObserver implementations in my secondary index creator.
> 
> the situation I got from hbase is this log.
> 
> ------------
> 12/08/13 10:37:08 WARN 
client.HConnectionManager$HConnectionImplementation: Failed all from 
region=blog,user6447991910946051755,1344821177585.7d4cbd4a9817ab7cb5c6219498d854a4.,
 
hostname=search-ddm-test5, port=60020
> java.util.concurrent.ExecutionException: 
java.net.SocketTimeoutException: Call to 
search-ddm-test5/xx.xx.xx.xx:60020 failed on socket timeout exception: 
java.net.SocketTimeoutException: 60000 millis timeout while waiting for 
channel to be ready for read. ch : 
java.nio.channels.SocketChannel[connected local=/xx.xx.xx.xx:53733 
remote=search-ddm-test5/xx.xx.xx.xx:60020]
>        at 
java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
>        at java.util.concurrent.FutureTask.get(FutureTask.java:83)
>        at 
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.processBatchCallback(HConnectionManager.java:1557)
>        at 
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.processBatch(HConnectionManager.java:1409)
>        at 
org.apache.hadoop.hbase.client.HTable.flushCommits(HTable.java:943)
>        at org.apache.hadoop.hbase.client.HTable.doPut(HTable.java:820)
>        at org.apache.hadoop.hbase.client.HTable.put(HTable.java:795)
>        at com.yahoo.ycsb.db.HBaseClient.update(HBaseClient.java:321)
>        at com.yahoo.ycsb.DBWrapper.update(DBWrapper.java:126)
>        at 
com.yahoo.ycsb.workloads.CoreWorkload.doTransactionUpdate(CoreWorkload.java:628)
>        at 
com.yahoo.ycsb.workloads.CoreWorkload.doTransaction(CoreWorkload.java:483)
>        at com.yahoo.ycsb.ClientThread.run(Client.java:233)
> Caused by: java.net.SocketTimeoutException: Call to 
search-ddm-test5/10.11.239.220:60020 failed on socket timeout exception: 
java.net.SocketTimeoutException: 60000 millis timeout while waiting for 
channel to be ready for read. ch : 
java.nio.channels.SocketChannel[connected local=/xx.xx.xx.xx:53733 
remote=search-ddm-test5/xx.xx.xx.xx:60020]
>        at 
org.apache.hadoop.hbase.ipc.HBaseClient.wrapException(HBaseClient.java:949)
>        at 
org.apache.hadoop.hbase.ipc.HBaseClient.call(HBaseClient.java:922)
>        at 
org.apache.hadoop.hbase.ipc.WritableRpcEngine$Invoker.invoke(WritableRpcEngine.java:150)
>        at $Proxy5.multi(Unknown Source)
>        at 
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation$3$1.call(HConnectionManager.java:1386)
>        at 
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation$3$1.call(HConnectionManager.java:1384)
>        at 
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.getRegionServerWithoutRetries(HConnectionManager.java:1365)
>        at 
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation$3.call(HConnectionManager.java:1383)
>        at 
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation$3.call(HConnectionManager.java:1381)
>        at 
java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>        at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>        at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>        at java.lang.Thread.run(Thread.java:662)
> Caused by: java.net.SocketTimeoutException: 60000 millis timeout while 
waiting for channel to be ready for read. ch : 
java.nio.channels.SocketChannel[connected local=/xx.xx.xx.xx:53733 
remote=search-ddm-test5/xx.xx.xx.xx:60020]
>        at 
org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:164)
>        at 
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
>        at 
org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
>        at java.io.FilterInputStream.read(FilterInputStream.java:116)
>        at 
org.apache.hadoop.hbase.ipc.HBaseClient$Connection$PingInputStream.read(HBaseClient.java:311)
>        at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
>        at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
>        at java.io.DataInputStream.readInt(DataInputStream.java:370)
>        at 
org.apache.hadoop.hbase.ipc.HBaseClient$Connection.receiveResponse(HBaseClient.java:571)
>        at 
org.apache.hadoop.hbase.ipc.HBaseClient$Connection.run(HBaseClient.java:505)
> 200 sec: 237725 operations; 0 current ops/sec;
> 210 sec: 237725 operations; 0 current ops/sec;
> 220 sec: 237725 operations; 0 current ops/sec;
> ….
> ….
> …
> 560 sec: 237725 operations; 0 current ops/sec;
> 570 sec: 237725 operations; 0 current ops/sec;
> 580 sec: 237725 operations; 0 current ops/sec;
> 590 sec: 237725 operations; 0 current ops/sec;
> 600 sec: 237725 operations; 0 current ops/sec;
> 610 sec: 237725 operations; 0 current ops/sec;
> 620 sec: 237725 operations; 0 current ops/sec;
> 630 sec: 237725 operations; 0 current ops/sec;
> 640 sec: 237725 operations; 0 current ops/sec;
> 650 sec: 237725 operations; 0 current ops/sec;
> 660 sec: 237725 operations; 0 current ops/sec;
> 670 sec: 237725 operations; 0 current ops/sec;
> 680 sec: 237725 operations; 0 current ops/sec;
> 690 sec: 237725 operations; 0 current ops/sec;
> 700 sec: 237725 operations; 0 current ops/sec;
> 710 sec: 237725 operations; 0 current ops/sec;
> 720 sec: 237725 operations; 0 current ops/sec;
> 730 sec: 237725 operations; 0 current ops/sec;
> 740 sec: 237725 operations; 0 current ops/sec;
> 750 sec: 237725 operations; 0 current ops/sec;
> 760 sec: 237725 operations; 0 current ops/sec;
> ------------
> 
> there was several normal insertions about 10000 TPS.
> but, after this operations, no more insertions, just warnings from 
hbase.
> 
> in my implementations,
> I didn't use HTablePool, Counter, only thing I used is just to put the 
mapping information into a secondary index table.
> 
> so, the question is.
> 1. does coprocessors not reliable yet?
> 2. how does it's performance?
> 3. is there any references which can help us to make it safe usages?
> 
> thanks for your concerns.
> 
> - Henry


Reply via email to