You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by 陈加俊 <cj...@gmail.com> on 2011/04/24 08:56:35 UTC

why merge regions error

I run jruby as follows :

# Name of this script
NAME = "merge_table"

# Print usage for this script
def usage
  puts 'Usage: %s.rb TABLE_NAME' % NAME
  exit!
end

# Get configuration to use.
c = HBaseConfiguration.new()

# Set hadoop filesystem configuration using the hbase.rootdir.
# Otherwise, we'll always use localhost though the hbase.rootdir
# might be pointing at hdfs location.
c.set("fs.default.name", c.get(HConstants::HBASE_DIR))
fs = FileSystem.get(c)

# Get a logger and a metautils instance.
LOG = LogFactory.getLog(NAME)

# Check arguments
if ARGV.size < 1 || ARGV.size > 2
  usage
end

tableName = java.lang.String.new(ARGV[0])
LOG.info("merge regions of table: " + tableName.toString())

HMerge.merge(c,fs,Bytes.toBytes(tableName))


But it is strange that :

......
11/04/24 14:55:31 DEBUG client.HConnectionManager$TableServers: Cache hit
for row <> in tableName .META.: location server 192.168.0.185:60020,
location region name .META.,,1
11/04/24 14:55:31 INFO hbase.HMerge: Row:
<ContentPageTempletValidate,,1289377363886>
11/04/24 14:55:31 INFO hbase.HMerge: Row: <EnterNodeRNDV,,1301653657945>
11/04/24 14:55:31 INFO hbase.HMerge: only one region - nothing to merge
11/04/24 14:55:31 DEBUG regionserver.HLog: closing hlog writer in hdfs://
test.namenode.uc.uuwatch.com:9000/hbase/cjjPutTest1/merge_1303628131187.logs
11/04/24 14:55:31 INFO zookeeper.ZooKeeper: Session: 0x32f5daa5e9700b3
closed
11/04/24 14:55:31 INFO zookeeper.ClientCnxn: EventThread shut down
11/04/24 14:55:31 DEBUG zookeeper.ZooKeeperWrapper: Closed connection with
ZooKeeper

the table cjjPutTest1 has 166 regions ,Why it said only one region ?
-- 
Thanks & Best regards
jiajun

Re: why merge regions error

Posted by Stack <st...@duboce.net>.
How can we help you?
St.Ack

On Sun, Apr 24, 2011 at 2:58 AM, 陈加俊 <cj...@gmail.com> wrote:
> It should :
>
> OnlineMerger(final HBaseConfiguration conf, final FileSystem fs, final
> byte[] tableName) throws IOException
>        {
>            super(conf, fs, tableName);
>            this.tableName = tableName;
>            table = new HTable(conf, META_TABLE_NAME);
>            final Filter filter = new PrefixFilter(this.tableName);
>            final Scan scan = new Scan();
>            scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
>            scan.setFilter(filter);
>            metaScanner = table.getScanner(scan);
>            latestRegion = null;
>        }
>
> or  like this :
>
>  private HRegionInfo nextRegion() throws IOException
>        {
>            try
>            {
>                //FIXME
>                for (;;)
>                {
>                    final Result results = getMetaRow();
>                    if (results == null)
>                    {
>                        return null;
>                    }
>                    final byte[] regionInfoValue =
> results.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
>                    if (regionInfoValue == null || regionInfoValue.length ==
> 0)
>                    {
>                        throw new NoSuchElementException("meta region entry
> missing " + Bytes.toString(CATALOG_FAMILY)
>                                + ":" +
> Bytes.toString(REGIONINFO_QUALIFIER));
>                    }
>                    final HRegionInfo region =
> Writables.getHRegionInfo(regionInfoValue);
>                    //FIXME
>                    if (!Bytes.equals(region.getTableDesc().getName(),
> tableName))
>                    {
>                        continue;
>                    }
>
>                    checkOfflined(region);
>                    return region;
>                }
>            }
>            catch (IOException e)
>            {
>                e = RemoteExceptionHandler.checkIOException(e);
>                LOG.error("meta scanner error", e);
>                metaScanner.close();
>                throw e;
>            }
>        }
>
> On Sun, Apr 24, 2011 at 3:32 PM, 陈加俊 <cj...@gmail.com> wrote:
>
>>  @Override
>>     protected HRegionInfo[] next() throws IOException {
>>       List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
>>       if(latestRegion == null) {
>>         latestRegion = nextRegion();
>>       }
>>       if(latestRegion != null) {
>>         regions.add(latestRegion);
>>       }
>>       latestRegion = nextRegion();
>>       if(latestRegion != null) {
>>         regions.add(latestRegion);
>>       }
>>       return regions.toArray(new HRegionInfo[regions.size()]);
>>     }
>>
>> the method nextRegion may get Null ,So there is a bug in HBase0.20.6.
>>
>>
>>
>> On Sun, Apr 24, 2011 at 2:56 PM, 陈加俊 <cj...@gmail.com> wrote:
>>
>>> I run jruby as follows :
>>>
>>> # Name of this script
>>> NAME = "merge_table"
>>>
>>> # Print usage for this script
>>> def usage
>>>   puts 'Usage: %s.rb TABLE_NAME' % NAME
>>>   exit!
>>> end
>>>
>>> # Get configuration to use.
>>> c = HBaseConfiguration.new()
>>>
>>> # Set hadoop filesystem configuration using the hbase.rootdir.
>>> # Otherwise, we'll always use localhost though the hbase.rootdir
>>> # might be pointing at hdfs location.
>>> c.set("fs.default.name", c.get(HConstants::HBASE_DIR))
>>> fs = FileSystem.get(c)
>>>
>>> # Get a logger and a metautils instance.
>>> LOG = LogFactory.getLog(NAME)
>>>
>>> # Check arguments
>>> if ARGV.size < 1 || ARGV.size > 2
>>>   usage
>>> end
>>>
>>> tableName = java.lang.String.new(ARGV[0])
>>> LOG.info("merge regions of table: " + tableName.toString())
>>>
>>> HMerge.merge(c,fs,Bytes.toBytes(tableName))
>>>
>>>
>>> But it is strange that :
>>>
>>> ......
>>> 11/04/24 14:55:31 DEBUG client.HConnectionManager$TableServers: Cache hit
>>> for row <> in tableName .META.: location server 192.168.0.185:60020,
>>> location region name .META.,,1
>>> 11/04/24 14:55:31 INFO hbase.HMerge: Row:
>>> <ContentPageTempletValidate,,1289377363886>
>>> 11/04/24 14:55:31 INFO hbase.HMerge: Row: <EnterNodeRNDV,,1301653657945>
>>> 11/04/24 14:55:31 INFO hbase.HMerge: only one region - nothing to merge
>>> 11/04/24 14:55:31 DEBUG regionserver.HLog: closing hlog writer in hdfs://
>>> test.namenode.uc.uuwatch.com:9000/hbase/cjjPutTest1/merge_1303628131187.logs
>>> 11/04/24 14:55:31 INFO zookeeper.ZooKeeper: Session: 0x32f5daa5e9700b3
>>> closed
>>> 11/04/24 14:55:31 INFO zookeeper.ClientCnxn: EventThread shut down
>>> 11/04/24 14:55:31 DEBUG zookeeper.ZooKeeperWrapper: Closed connection with
>>> ZooKeeper
>>>
>>> the table cjjPutTest1 has 166 regions ,Why it said only one region ?
>>> --
>>> Thanks & Best regards
>>> jiajun
>>>
>>>
>>
>>
>> --
>> Thanks & Best regards
>> jiajun
>>
>>
>
>
> --
> Thanks & Best regards
> jiajun
>

Re: why merge regions error

Posted by 陈加俊 <cj...@gmail.com>.
It should :

OnlineMerger(final HBaseConfiguration conf, final FileSystem fs, final
byte[] tableName) throws IOException
        {
            super(conf, fs, tableName);
            this.tableName = tableName;
            table = new HTable(conf, META_TABLE_NAME);
            final Filter filter = new PrefixFilter(this.tableName);
            final Scan scan = new Scan();
            scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
            scan.setFilter(filter);
            metaScanner = table.getScanner(scan);
            latestRegion = null;
        }

or  like this :

 private HRegionInfo nextRegion() throws IOException
        {
            try
            {
                //FIXME
                for (;;)
                {
                    final Result results = getMetaRow();
                    if (results == null)
                    {
                        return null;
                    }
                    final byte[] regionInfoValue =
results.getValue(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
                    if (regionInfoValue == null || regionInfoValue.length ==
0)
                    {
                        throw new NoSuchElementException("meta region entry
missing " + Bytes.toString(CATALOG_FAMILY)
                                + ":" +
Bytes.toString(REGIONINFO_QUALIFIER));
                    }
                    final HRegionInfo region =
Writables.getHRegionInfo(regionInfoValue);
                    //FIXME
                    if (!Bytes.equals(region.getTableDesc().getName(),
tableName))
                    {
                        continue;
                    }

                    checkOfflined(region);
                    return region;
                }
            }
            catch (IOException e)
            {
                e = RemoteExceptionHandler.checkIOException(e);
                LOG.error("meta scanner error", e);
                metaScanner.close();
                throw e;
            }
        }

On Sun, Apr 24, 2011 at 3:32 PM, 陈加俊 <cj...@gmail.com> wrote:

>  @Override
>     protected HRegionInfo[] next() throws IOException {
>       List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
>       if(latestRegion == null) {
>         latestRegion = nextRegion();
>       }
>       if(latestRegion != null) {
>         regions.add(latestRegion);
>       }
>       latestRegion = nextRegion();
>       if(latestRegion != null) {
>         regions.add(latestRegion);
>       }
>       return regions.toArray(new HRegionInfo[regions.size()]);
>     }
>
> the method nextRegion may get Null ,So there is a bug in HBase0.20.6.
>
>
>
> On Sun, Apr 24, 2011 at 2:56 PM, 陈加俊 <cj...@gmail.com> wrote:
>
>> I run jruby as follows :
>>
>> # Name of this script
>> NAME = "merge_table"
>>
>> # Print usage for this script
>> def usage
>>   puts 'Usage: %s.rb TABLE_NAME' % NAME
>>   exit!
>> end
>>
>> # Get configuration to use.
>> c = HBaseConfiguration.new()
>>
>> # Set hadoop filesystem configuration using the hbase.rootdir.
>> # Otherwise, we'll always use localhost though the hbase.rootdir
>> # might be pointing at hdfs location.
>> c.set("fs.default.name", c.get(HConstants::HBASE_DIR))
>> fs = FileSystem.get(c)
>>
>> # Get a logger and a metautils instance.
>> LOG = LogFactory.getLog(NAME)
>>
>> # Check arguments
>> if ARGV.size < 1 || ARGV.size > 2
>>   usage
>> end
>>
>> tableName = java.lang.String.new(ARGV[0])
>> LOG.info("merge regions of table: " + tableName.toString())
>>
>> HMerge.merge(c,fs,Bytes.toBytes(tableName))
>>
>>
>> But it is strange that :
>>
>> ......
>> 11/04/24 14:55:31 DEBUG client.HConnectionManager$TableServers: Cache hit
>> for row <> in tableName .META.: location server 192.168.0.185:60020,
>> location region name .META.,,1
>> 11/04/24 14:55:31 INFO hbase.HMerge: Row:
>> <ContentPageTempletValidate,,1289377363886>
>> 11/04/24 14:55:31 INFO hbase.HMerge: Row: <EnterNodeRNDV,,1301653657945>
>> 11/04/24 14:55:31 INFO hbase.HMerge: only one region - nothing to merge
>> 11/04/24 14:55:31 DEBUG regionserver.HLog: closing hlog writer in hdfs://
>> test.namenode.uc.uuwatch.com:9000/hbase/cjjPutTest1/merge_1303628131187.logs
>> 11/04/24 14:55:31 INFO zookeeper.ZooKeeper: Session: 0x32f5daa5e9700b3
>> closed
>> 11/04/24 14:55:31 INFO zookeeper.ClientCnxn: EventThread shut down
>> 11/04/24 14:55:31 DEBUG zookeeper.ZooKeeperWrapper: Closed connection with
>> ZooKeeper
>>
>> the table cjjPutTest1 has 166 regions ,Why it said only one region ?
>> --
>> Thanks & Best regards
>> jiajun
>>
>>
>
>
> --
> Thanks & Best regards
> jiajun
>
>


-- 
Thanks & Best regards
jiajun

Re: why merge regions error

Posted by 陈加俊 <cj...@gmail.com>.
 @Override
    protected HRegionInfo[] next() throws IOException {
      List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
      if(latestRegion == null) {
        latestRegion = nextRegion();
      }
      if(latestRegion != null) {
        regions.add(latestRegion);
      }
      latestRegion = nextRegion();
      if(latestRegion != null) {
        regions.add(latestRegion);
      }
      return regions.toArray(new HRegionInfo[regions.size()]);
    }

the method nextRegion may get Null ,So there is a bug in HBase0.20.6.



On Sun, Apr 24, 2011 at 2:56 PM, 陈加俊 <cj...@gmail.com> wrote:

> I run jruby as follows :
>
> # Name of this script
> NAME = "merge_table"
>
> # Print usage for this script
> def usage
>   puts 'Usage: %s.rb TABLE_NAME' % NAME
>   exit!
> end
>
> # Get configuration to use.
> c = HBaseConfiguration.new()
>
> # Set hadoop filesystem configuration using the hbase.rootdir.
> # Otherwise, we'll always use localhost though the hbase.rootdir
> # might be pointing at hdfs location.
> c.set("fs.default.name", c.get(HConstants::HBASE_DIR))
> fs = FileSystem.get(c)
>
> # Get a logger and a metautils instance.
> LOG = LogFactory.getLog(NAME)
>
> # Check arguments
> if ARGV.size < 1 || ARGV.size > 2
>   usage
> end
>
> tableName = java.lang.String.new(ARGV[0])
> LOG.info("merge regions of table: " + tableName.toString())
>
> HMerge.merge(c,fs,Bytes.toBytes(tableName))
>
>
> But it is strange that :
>
> ......
> 11/04/24 14:55:31 DEBUG client.HConnectionManager$TableServers: Cache hit
> for row <> in tableName .META.: location server 192.168.0.185:60020,
> location region name .META.,,1
> 11/04/24 14:55:31 INFO hbase.HMerge: Row:
> <ContentPageTempletValidate,,1289377363886>
> 11/04/24 14:55:31 INFO hbase.HMerge: Row: <EnterNodeRNDV,,1301653657945>
> 11/04/24 14:55:31 INFO hbase.HMerge: only one region - nothing to merge
> 11/04/24 14:55:31 DEBUG regionserver.HLog: closing hlog writer in hdfs://
> test.namenode.uc.uuwatch.com:9000/hbase/cjjPutTest1/merge_1303628131187.logs
> 11/04/24 14:55:31 INFO zookeeper.ZooKeeper: Session: 0x32f5daa5e9700b3
> closed
> 11/04/24 14:55:31 INFO zookeeper.ClientCnxn: EventThread shut down
> 11/04/24 14:55:31 DEBUG zookeeper.ZooKeeperWrapper: Closed connection with
> ZooKeeper
>
> the table cjjPutTest1 has 166 regions ,Why it said only one region ?
> --
> Thanks & Best regards
> jiajun
>
>


-- 
Thanks & Best regards
jiajun