Hi Edward. You still have DFS issues. Have you
followed the DFS related troubleshooting tips on the
troubleshooting page of the HBase wiki? Have you
considered adding more data nodes?

   - Andy

> From: Edward J. Yoon 
> Subject: Re: Force split
> To: [email protected]
> Date: Wednesday, April 29, 2009, 2:55 AM
> I found below error message. :(
> 
> 2009-04-29 16:16:57,380 WARN
> org.apache.hadoop.hdfs.DFSClient: Could
> not get block locations. Aborting...
> 2009-04-29 16:16:57,390 ERROR
> org.apache.hadoop.hbase.regionserver.CompactSplitThread:
> Compaction/Split failed for region
> SparseMatrix_randufgse,000000000002519,1240989321368
> java.io.IOException: Could not read from stream
>         at
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:119)
>         at
> java.io.DataInputStream.readByte(DataInputStream.java:248)
>         at
> org.apache.hadoop.io.WritableUtils.readVLong(WritableUtils.java:325)
>         at
> org.apache.hadoop.io.WritableUtils.readVInt(WritableUtils.java:346)
>         at
> org.apache.hadoop.io.Text.readString(Text.java:400)
>         at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.createBlockOutputStream(DFSClient.java:2779)
>         at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(DFSClient.java:2704)
>         at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(DFSClient.java:1997)
>         at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2183)
> 2009-04-29 16:18:46,051 INFO
> org.apache.hadoop.io.compress.CodecPool:
> Got brand-new decompressor
> 2009-04-29 16:18:46,051 INFO
> org.apache.hadoop.io.compress.CodecPool:
> Got brand-new decompressor
> 
> 
> On Wed, Apr 29, 2009 at 6:17 PM, Edward J. Yoon
> <[email protected]> wrote:
> > According to my understanding, there is no any code
> for
> > MODIFY_TABLE_SPLIT and table.jsp work as below.
> >
> > Writable[] arr = new Writable[1];
> > arr[0] = new
> ImmutableBytesWritable(Bytes.toBytes(key));
> > master.modifyTable(Bytes.toBytes(tableName),
> > HConstants.MODIFY_TABLE_SPLIT, arr);
> >
> > Anyway, I'd like to know how to force split.
> > Below is the info of 'Regions in randufgse'
> page.
> > Rows are sequntial numbers between 0 and 10,000.
> >
> > If I want to split by "000000000007648"
> then,
> >
> > master.modifyTable(Bytes.toBytes(tableName),
> > HConstants.MODIFY_TABLE_SPLIT, new
> >
> ImmutableBytesWritable(Bytes.toBytes("000000000007648"));
> >
> > Is enough?
> >
> > d8g055.nhncorp.com:60020:  , 000000000000465
> > a50891.nhncorp.com:60020: 000000000000465,
> 000000000000811
> > a51019.nhncorp.com:60020: 000000000000811,
> 000000000001134
> > a51019.nhncorp.com:60020: 000000000001134,
> 000000000001460
> > a51010.nhncorp.com:60020: 000000000001460,
> 000000000001723
> > a51010.nhncorp.com:60020: 000000000001723,
> 000000000001988
> > a51018.nhncorp.com:60020: 000000000001988,
> 000000000002254
> > a51018.nhncorp.com:60020: 000000000002254,
> 000000000002519
> > a50837.nhncorp.com:60020: 000000000002519,
> 000000000004648
> > a50837.nhncorp.com:60020: 000000000004648,
> >
> > On Wed, Apr 29, 2009 at 5:11 PM, Ryan Rawson
> <[email protected]> wrote:
> >> It works, it has worked in the past...
> >>
> >> One thing to remember is you can't always
> split...  You cant split until all
> >> the open scanners are closed.  Furthermore you
> can't split if a region has
> >> just split.  The reason for this is because once
> you split a region, files
> >> aren't rewritten, but the 2 daughter regions
> hold references to the parent
> >> region.  Until those regions are resolved via
> compaction (slow, minutes
> >> possibly), you can't split those daughter
> regions.
> >>
> >> So check again, maybe your splits are working
> after all.
> >>
> >> If you are doing imports, check out my
> Randomize.java i posted to the list
> >> last week.  That helps import performance
> substantially.
> >>
> >> On Wed, Apr 29, 2009 at 12:52 AM, Edward J. Yoon
> <[email protected]>wrote:
> >>
> >>> Hi,
> >>>
> >>> I saw that function on the web UI but it seems
> not implemented yet.
> >>>
> >>> public void modifyTable(final byte[]
> tableName, int op, Writable[] args)
> >>>  switch (op) {
> >>>    ...
> >>>    case MODIFY_TABLE_SPLIT:
> >>>    case MODIFY_TABLE_COMPACT:
> >>>    ..
> >>>  }
> >>> }
> >>>
> >>> Am I right? Pls let me know if i'm wrong.
> >>> --
> >>> Best Regards, Edward J. Yoon @ NHN, corp.
> >>> [email protected]
> >>> http://blog.udanax.org
> >>>
> >>
> >
> >
> >
> > --
> > Best Regards, Edward J. Yoon @ NHN, corp.
> > [email protected]
> > http://blog.udanax.org
> >
> 
> 
> 
> -- 
> Best Regards, Edward J. Yoon @ NHN, corp.
> [email protected]
> http://blog.udanax.org



Reply via email to