[ 
https://issues.apache.org/jira/browse/HBASE-11817?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14109256#comment-14109256
 ] 

Jean-Marc Spaggiari edited comment on HBASE-11817 at 8/25/14 4:21 PM:
----------------------------------------------------------------------

I ran the following code:

{code}
public class IncrementsAndSplits {

  public static void main(String[] args) throws MasterNotRunningException, 
ZooKeeperConnectionException, IOException, InterruptedException {
    Configuration conf = HBaseConfiguration.create();
    HConnection connection = HConnectionManager.createConnection(conf);
    HTableInterface table = connection.getTable("t1");
    byte[] rowKey = new byte[1];
    for (int i=0;i<0xffff;i++){
     ArrayList<Increment> operations = new ArrayList<Increment>();
     for (byte c1 = (byte)'A'; c1<=(byte)'Z'; c1++) {
       rowKey[0] = c1;
       Increment opIncr = new Increment(rowKey);
       opIncr.addColumn(Bytes.toBytes("f1"), HConstants.EMPTY_BYTE_ARRAY, 1);
       operations.add(opIncr);
     }
     table.batch(operations, null);
    }
    HBaseAdmin admin = new HBaseAdmin(conf);
    for (byte c1 = (byte)'A'; c1<=(byte)'Z'; c1++) {
     try { Thread.sleep(2000L); } catch (InterruptedException iex) {}
     rowKey[0] = c1;
     admin.split(Bytes.toBytes("t1"), rowKey);
    }
  }
}
{code}

And got the following result:
{code}
hbase(main):029:0> scan 't1'
ROW                                                       COLUMN+CELL           
                                          
 A    column=f1:, timestamp=1408980820902, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFF                                         
 B    column=f1:, timestamp=1408980820904, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFF                                         
 C    column=f1:, timestamp=1408980820906, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFF                                         
 D    column=f1:, timestamp=1408980820908, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFF                                         
 E    column=f1:, timestamp=1408980820909, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFF                                         
 F    column=f1:, timestamp=1408980820911, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFF                                         
 G    column=f1:, timestamp=1408980820913, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFF                                         
 H    column=f1:, timestamp=1408980820914, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFF                                         
 I    column=f1:, timestamp=1408980820916, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFF                                         
 J    column=f1:, timestamp=1408980820918, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFF                                         
 K    column=f1:, timestamp=1408980820919, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFF                                         
 L    column=f1:, timestamp=1408980820921, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFE                                         
 M    column=f1:, timestamp=1408980820923, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFE                                         
 N    column=f1:, timestamp=1408980820924, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFE                                         
 O    column=f1:, timestamp=1408980820926, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFE                                         
 P    column=f1:, timestamp=1408980820928, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFE                                         
 Q    column=f1:, timestamp=1408980820929, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFE                                         
 R    column=f1:, timestamp=1408980820931, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFE                                         
 S    column=f1:, timestamp=1408980820933, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFE                                         
 T    column=f1:, timestamp=1408980820934, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFE                                         
 U    column=f1:, timestamp=1408980820936, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFE                                         
 V    column=f1:, timestamp=1408980820938, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFE                                         
 W    column=f1:, timestamp=1408980820939, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFE                                         
 X    column=f1:, timestamp=1408980820941, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFE                                         
 Y    column=f1:, timestamp=1408980820943, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFE                                         
 Z    column=f1:, timestamp=1408980820944, 
value=\x00\x00\x00\x00\x00\x00\xFF\xFE                                         
26 row(s) in 0.7620 seconds
{code}

I tried with only 255 increments and went well. I will retry many different 
other scenarios to try to understand where it comes from.


was (Author: jmspaggi):
I ran the following code:

{code}
public class IncrementsAndSplits {

  public static void main(String[] args) throws MasterNotRunningException, 
ZooKeeperConnectionException, IOException, InterruptedException {
    Configuration conf = HBaseConfiguration.create();
    HConnection connection = HConnectionManager.createConnection(conf);
    HTableInterface table = connection.getTable("t1");
    byte[] rowKey = new byte[1];
    for (int i=0;i<0xffff;i++){
     ArrayList<Increment> operations = new ArrayList<Increment>();
     for (byte c1 = (byte)'A'; c1<=(byte)'Z'; c1++) {
       rowKey[0] = c1;
       Increment opIncr = new Increment(rowKey);
       opIncr.addColumn(Bytes.toBytes("f1"), HConstants.EMPTY_BYTE_ARRAY, 1);
       operations.add(opIncr);
     }
     table.batch(operations, null);
    }
    HBaseAdmin admin = new HBaseAdmin(conf);
    for (byte c1 = (byte)'A'; c1<=(byte)'Z'; c1++) {
     try { Thread.sleep(2000L); } catch (InterruptedException iex) {}
     rowKey[0] = c1;
     admin.split(Bytes.toBytes("t1"), rowKey);
    }
  }
}
{code}

And got the following result:
{code}
hbase(main):029:0> scan 't1'
ROW                                                       COLUMN+CELL           
                                                                                
                                                                  
 A                                                        column=f1:, 
timestamp=1408980820902, value=\x00\x00\x00\x00\x00\x00\xFF\xFF                 
                                                                            
 B                                                        column=f1:, 
timestamp=1408980820904, value=\x00\x00\x00\x00\x00\x00\xFF\xFF                 
                                                                            
 C                                                        column=f1:, 
timestamp=1408980820906, value=\x00\x00\x00\x00\x00\x00\xFF\xFF                 
                                                                            
 D                                                        column=f1:, 
timestamp=1408980820908, value=\x00\x00\x00\x00\x00\x00\xFF\xFF                 
                                                                            
 E                                                        column=f1:, 
timestamp=1408980820909, value=\x00\x00\x00\x00\x00\x00\xFF\xFF                 
                                                                            
 F                                                        column=f1:, 
timestamp=1408980820911, value=\x00\x00\x00\x00\x00\x00\xFF\xFF                 
                                                                            
 G                                                        column=f1:, 
timestamp=1408980820913, value=\x00\x00\x00\x00\x00\x00\xFF\xFF                 
                                                                            
 H                                                        column=f1:, 
timestamp=1408980820914, value=\x00\x00\x00\x00\x00\x00\xFF\xFF                 
                                                                            
 I                                                        column=f1:, 
timestamp=1408980820916, value=\x00\x00\x00\x00\x00\x00\xFF\xFF                 
                                                                            
 J                                                        column=f1:, 
timestamp=1408980820918, value=\x00\x00\x00\x00\x00\x00\xFF\xFF                 
                                                                            
 K                                                        column=f1:, 
timestamp=1408980820919, value=\x00\x00\x00\x00\x00\x00\xFF\xFF                 
                                                                            
 L                                                        column=f1:, 
timestamp=1408980820921, value=\x00\x00\x00\x00\x00\x00\xFF\xFE                 
                                                                            
 M                                                        column=f1:, 
timestamp=1408980820923, value=\x00\x00\x00\x00\x00\x00\xFF\xFE                 
                                                                            
 N                                                        column=f1:, 
timestamp=1408980820924, value=\x00\x00\x00\x00\x00\x00\xFF\xFE                 
                                                                            
 O                                                        column=f1:, 
timestamp=1408980820926, value=\x00\x00\x00\x00\x00\x00\xFF\xFE                 
                                                                            
 P                                                        column=f1:, 
timestamp=1408980820928, value=\x00\x00\x00\x00\x00\x00\xFF\xFE                 
                                                                            
 Q                                                        column=f1:, 
timestamp=1408980820929, value=\x00\x00\x00\x00\x00\x00\xFF\xFE                 
                                                                            
 R                                                        column=f1:, 
timestamp=1408980820931, value=\x00\x00\x00\x00\x00\x00\xFF\xFE                 
                                                                            
 S                                                        column=f1:, 
timestamp=1408980820933, value=\x00\x00\x00\x00\x00\x00\xFF\xFE                 
                                                                            
 T                                                        column=f1:, 
timestamp=1408980820934, value=\x00\x00\x00\x00\x00\x00\xFF\xFE                 
                                                                            
 U                                                        column=f1:, 
timestamp=1408980820936, value=\x00\x00\x00\x00\x00\x00\xFF\xFE                 
                                                                            
 V                                                        column=f1:, 
timestamp=1408980820938, value=\x00\x00\x00\x00\x00\x00\xFF\xFE                 
                                                                            
 W                                                        column=f1:, 
timestamp=1408980820939, value=\x00\x00\x00\x00\x00\x00\xFF\xFE                 
                                                                            
 X                                                        column=f1:, 
timestamp=1408980820941, value=\x00\x00\x00\x00\x00\x00\xFF\xFE                 
                                                                            
 Y                                                        column=f1:, 
timestamp=1408980820943, value=\x00\x00\x00\x00\x00\x00\xFF\xFE                 
                                                                            
 Z                                                        column=f1:, 
timestamp=1408980820944, value=\x00\x00\x00\x00\x00\x00\xFF\xFE                 
                                                                            
26 row(s) in 0.7620 seconds
{code}

I tried with only 255 increments and went well. I will retry many different 
other scenarios to try to understand where it comes from.

> HTable.batch() loses operations when region is splited
> ------------------------------------------------------
>
>                 Key: HBASE-11817
>                 URL: https://issues.apache.org/jira/browse/HBASE-11817
>             Project: HBase
>          Issue Type: Bug
>          Components: Admin, Client
>    Affects Versions: 0.98.4
>         Environment: 0.98.4+hadoop 2.4.1, 0.98.4 stand-alone, jdk1.6
>            Reporter: t samkawa
>
> Using HTable.batch() often loses increment operation when region split ran.
> Test code snpipet is below; 
> Running this 2 code blocks concurrently, different values were often 
> recoreded although all value should be same 0xffff.
> {code}
> // --- code 1 ---
> HTable table = new HTable(CONF);
> byte[] rowKey = new byte[1];
> for (int i=0;i<0xffff;i++){
>  ArrayList<Increment> operations = new ArrayList<Increment>();
>  for (byte c1 = (byte)'A'; c1<=(byte)'Z'; c1++) {
>    rowKey[0] = c1;
>    Increment opIncr = new Increment(rowKey);
>    opIncr.addColumn(FAM, HConstants.EMPTY_BYTE_ARRAY, 1);
>    operations.add(opIncr);
>  }
>  table.batch(operations, null);
> }
> // -- code2 --
> HBaseAdmin admin = new HBaseAdmin(CONF);
> byte[] rowKey = new byte[1];
> for (byte c1 = (byte)'A'; c1<=(byte)'Z'; c1++) {
>  try { Thread.sleep(2000L); } catch (InterruptedException iex) {}
>  rowKey[0] = c1;
>  admin.split(TABLE_NAME, rowKey);
> }
> /////
> {code}
> Using table.increment() instead of table.batch() works fine. But that change 
> gets slower . 



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to